1
0
Fork 0

Adding upstream version 1.34.4.

Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
Daniel Baumann 2025-05-24 07:26:29 +02:00
parent e393c3af3f
commit 4978089aab
Signed by: daniel
GPG key ID: FBB4F0E80A80222F
4963 changed files with 677545 additions and 0 deletions

View file

@ -0,0 +1,142 @@
# Intel RDT Input Plugin
This plugin collects information provided by monitoring features of the
[Intel Resource Director Technology][rdt], a hardware framework to monitor and
control the utilization of shared resources (e.g. last level cache,
memory bandwidth).
Intels Resource Director Technology (RDT) framework consists of:
- Cache Monitoring Technology (CMT)
- Memory Bandwidth Monitoring (MBM)
- Cache Allocation Technology (CAT)
- Code and Data Prioritization (CDP)
As multithreaded and multicore platform architectures emerge, the last level
cache and memory bandwidth are key resources to manage for running workloads in
single-threaded, multithreaded, or complex virtual machine environments. Intel
introduces CMT, MBM, CAT and CDP to manage these workloads across shared
resources.
⭐ Telegraf v1.16.0
🏷️ hardware, system
💻 linux, freebsd, macos
[rdt]: https://www.intel.com/content/www/us/en/architecture-and-technology/resource-director-technology.html
## Requirements
The plugin requires the `pqos` cli tool in version 4.0+ to be installed and
configured to work in `OS Interface` mode. The tool is part of the
[Intel(R) RDT Software Package][cmt_cat].
> [!IMPORTANT]
> The `pqos` binary needs to run as root. If telegraf is not running as root
> you need to enable sudo for `pqos` and set the `use_sudo` option to `true`.
To setup `pqos` correctly check the [installation guide][install]. For help on
how to configure the tool visit the [wiki][wiki] and read the
[resource control documentation][resctl]
[cmt_cat]: https://github.com/intel/intel-cmt-cat
[install]: https://github.com/intel/intel-cmt-cat/blob/master/INSTALL
[wiki]: https://github.com/intel/intel-cmt-cat/wiki
[resctl]: https://github.com/intel/intel-cmt-cat/wiki/resctrl
## Service Input <!-- @/docs/includes/service_input.md -->
This plugin is a service input. Normal plugins gather metrics determined by the
interval setting. Service plugins start a service to listen and wait for
metrics or events to occur. Service plugins have two key differences from
normal plugins:
1. The global or plugin specific `interval` setting may not apply
2. The CLI options of `--test`, `--test-wait`, and `--once` may not produce
output for this plugin
## Global configuration options <!-- @/docs/includes/plugin_config.md -->
In addition to the plugin-specific configuration settings, plugins support
additional global and plugin configuration settings. These settings are used to
modify metrics, tags, and field or create aliases and configure ordering, etc.
See the [CONFIGURATION.md][CONFIGURATION.md] for more details.
[CONFIGURATION.md]: ../../../docs/CONFIGURATION.md#plugins
## Configuration
```toml @sample.conf
# Read Intel RDT metrics
# This plugin ONLY supports non-Windows
[[inputs.intel_rdt]]
## Optionally set sampling interval to Nx100ms.
## This value is propagated to pqos tool. Interval format is defined by pqos itself.
## If not provided or provided 0, will be set to 10 = 10x100ms = 1s.
# sampling_interval = "10"
## Optionally specify the path to pqos executable.
## If not provided, auto discovery will be performed.
# pqos_path = "/usr/local/bin/pqos"
## Optionally specify if IPC and LLC_Misses metrics shouldn't be propagated.
## If not provided, default value is false.
# shortened_metrics = false
## Specify the list of groups of CPU core(s) to be provided as pqos input.
## Mandatory if processes aren't set and forbidden if processes are specified.
## e.g. ["0-3", "4,5,6"] or ["1-3,4"]
# cores = ["0-3"]
## Specify the list of processes for which Metrics will be collected.
## Mandatory if cores aren't set and forbidden if cores are specified.
## e.g. ["qemu", "pmd"]
# processes = ["process"]
## Specify if the pqos process should be called with sudo.
## Mandatory if the telegraf process does not run as root.
# use_sudo = false
```
## Troubleshooting
Pointing to non-existing cores will lead to throwing an error by _pqos_ and the
plugin will not work properly. Be sure to check provided core number exists
within desired system.
Be aware, reading Intel RDT metrics by _pqos_ cannot be done simultaneously on
the same resource. Do not use any other _pqos_ instance that is monitoring the
same cores or PIDs within the working system. It is not possible to monitor
same cores or PIDs on different groups.
PIDs associated for the given process could be manually checked by `pidof`
command. E.g:
```sh
pidof PROCESS
```
where `PROCESS` is process name.
## Metrics
| Name | Full name | Description |
|---------------|-----------------------------------------------|-------------|
| MBL | Memory Bandwidth on Local NUMA Node | Memory bandwidth utilization by the relevant CPU core/process on the local NUMA memory channel |
| MBR | Memory Bandwidth on Remote NUMA Node | Memory bandwidth utilization by the relevant CPU core/process on the remote NUMA memory channel |
| MBT | Total Memory Bandwidth | Total memory bandwidth utilized by a CPU core/process on local and remote NUMA memory channels |
| LLC | L3 Cache Occupancy | Total Last Level Cache occupancy by a CPU core/process |
| LLC_Misses* | L3 Cache Misses | Total Last Level Cache misses by a CPU core/process |
| IPC* | Instructions Per Cycle | Total instructions per cycle executed by a CPU core/process |
*optional
## Example Output
```text
rdt_metric,cores=12\,19,host=r2-compute-20,name=IPC,process=top value=0 1598962030000000000
rdt_metric,cores=12\,19,host=r2-compute-20,name=LLC_Misses,process=top value=0 1598962030000000000
rdt_metric,cores=12\,19,host=r2-compute-20,name=LLC,process=top value=0 1598962030000000000
rdt_metric,cores=12\,19,host=r2-compute-20,name=MBL,process=top value=0 1598962030000000000
rdt_metric,cores=12\,19,host=r2-compute-20,name=MBR,process=top value=0 1598962030000000000
rdt_metric,cores=12\,19,host=r2-compute-20,name=MBT,process=top value=0 1598962030000000000
```

View file

@ -0,0 +1,556 @@
//go:generate ../../../tools/readme_config_includer/generator
//go:build !windows
package intel_rdt
import (
"bufio"
"context"
_ "embed"
"errors"
"fmt"
"io"
"os"
"os/exec"
"regexp"
"strconv"
"strings"
"sync"
"syscall"
"time"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/internal/choice"
"github.com/influxdata/telegraf/plugins/inputs"
)
//go:embed sample.conf
var sampleConfig string
var pqosMetricOrder = map[int]string{
0: "IPC", // Instructions Per Cycle
1: "LLC_Misses", // Cache Misses
2: "LLC", // L3 Cache Occupancy
3: "MBL", // Memory Bandwidth on Local NUMA Node
4: "MBR", // Memory Bandwidth on Remote NUMA Node
5: "MBT", // Total Memory Bandwidth
}
const (
timestampFormat = "2006-01-02 15:04:05"
defaultSamplingInterval = 10
pqosInitOutputLinesNumber = 4
numberOfMetrics = 6
secondsDenominator = 10
)
type IntelRDT struct {
PqosPath string `toml:"pqos_path"`
Cores []string `toml:"cores"`
Processes []string `toml:"processes"`
SamplingInterval int32 `toml:"sampling_interval"`
ShortenedMetrics bool `toml:"shortened_metrics"`
UseSudo bool `toml:"use_sudo"`
Log telegraf.Logger `toml:"-"`
publisher publisher
processor processesHandler
stopPQOSChan chan bool
quitChan chan struct{}
errorChan chan error
parsedCores []string
processesPIDsMap map[string]string
cancel context.CancelFunc
wg sync.WaitGroup
}
type processMeasurement struct {
name string
measurement string
}
type splitCSVLine struct {
timeValue string
metricsValues []string
coreOrPIDsValues []string
}
func (*IntelRDT) SampleConfig() string {
return sampleConfig
}
func (r *IntelRDT) Start(acc telegraf.Accumulator) error {
ctx, cancel := context.WithCancel(context.Background())
r.cancel = cancel
r.processor = newProcessor()
r.publisher = newPublisher(acc, r.Log, r.ShortenedMetrics)
err := r.initialize()
if err != nil {
return err
}
r.publisher.publish(ctx)
go r.errorHandler(ctx)
go r.scheduler(ctx)
return nil
}
func (*IntelRDT) Gather(telegraf.Accumulator) error {
return nil
}
func (r *IntelRDT) Stop() {
r.cancel()
r.wg.Wait()
}
func (r *IntelRDT) initialize() error {
r.stopPQOSChan = make(chan bool)
r.quitChan = make(chan struct{})
r.errorChan = make(chan error)
err := validatePqosPath(r.PqosPath)
if err != nil {
return err
}
if len(r.Cores) != 0 && len(r.Processes) != 0 {
return errors.New("monitoring start error, process and core tracking can not be done simultaneously")
}
if len(r.Cores) == 0 && len(r.Processes) == 0 {
return errors.New("monitoring start error, at least one of cores or processes must be provided in config")
}
if r.SamplingInterval == 0 {
r.SamplingInterval = defaultSamplingInterval
}
if err := validateInterval(r.SamplingInterval); err != nil {
return err
}
r.parsedCores, err = parseCoresConfig(r.Cores)
if err != nil {
return err
}
r.processesPIDsMap, err = r.associateProcessesWithPIDs(r.Processes)
if err != nil {
return err
}
return nil
}
func (r *IntelRDT) errorHandler(ctx context.Context) {
r.wg.Add(1)
defer r.wg.Done()
for {
select {
case err := <-r.errorChan:
if err != nil {
r.Log.Error(fmt.Sprintf("Error: %v", err))
r.quitChan <- struct{}{}
}
case <-ctx.Done():
return
}
}
}
func (r *IntelRDT) scheduler(ctx context.Context) {
r.wg.Add(1)
defer r.wg.Done()
interval := time.Duration(r.SamplingInterval)
ticker := time.NewTicker(interval * time.Second / secondsDenominator)
r.createArgsAndStartPQOS(ctx)
for {
select {
case <-ticker.C:
if len(r.Processes) != 0 {
err := r.checkPIDsAssociation(ctx)
if err != nil {
r.errorChan <- err
}
}
case <-r.quitChan:
r.cancel()
return
case <-ctx.Done():
return
}
}
}
func (r *IntelRDT) checkPIDsAssociation(ctx context.Context) error {
newProcessesPIDsMap, err := r.associateProcessesWithPIDs(r.Processes)
if err != nil {
return err
}
// change in PIDs association appears
if !cmp.Equal(newProcessesPIDsMap, r.processesPIDsMap) {
r.Log.Warnf("PIDs association has changed. Refreshing...")
if len(r.processesPIDsMap) != 0 {
r.stopPQOSChan <- true
}
r.processesPIDsMap = newProcessesPIDsMap
r.createArgsAndStartPQOS(ctx)
}
return nil
}
func (r *IntelRDT) associateProcessesWithPIDs(providedProcesses []string) (map[string]string, error) {
availableProcesses, err := r.processor.getAllProcesses()
if err != nil {
return nil, errors.New("cannot gather information of all available processes")
}
mapProcessPIDs := make(map[string]string, len(availableProcesses))
for _, availableProcess := range availableProcesses {
if choice.Contains(availableProcess.Name, providedProcesses) {
pid := availableProcess.PID
mapProcessPIDs[availableProcess.Name] = mapProcessPIDs[availableProcess.Name] + strconv.Itoa(pid) + ","
}
}
for key := range mapProcessPIDs {
mapProcessPIDs[key] = strings.TrimSuffix(mapProcessPIDs[key], ",")
}
return mapProcessPIDs, nil
}
func (r *IntelRDT) createArgsAndStartPQOS(ctx context.Context) {
args := []string{"-r", "--iface-os", "--mon-file-type=csv", fmt.Sprintf("--mon-interval=%d", r.SamplingInterval)}
if len(r.parsedCores) != 0 {
coresArg := createArgCores(r.parsedCores)
args = append(args, coresArg)
go r.readData(ctx, args, nil)
} else if len(r.processesPIDsMap) != 0 {
processArg := createArgProcess(r.processesPIDsMap)
args = append(args, processArg)
go r.readData(ctx, args, r.processesPIDsMap)
}
}
func (r *IntelRDT) readData(ctx context.Context, args []string, processesPIDsAssociation map[string]string) {
r.wg.Add(1)
defer r.wg.Done()
cmd := exec.Command(r.PqosPath, args...)
if r.UseSudo {
// run pqos with `/bin/sh -c "sudo /path/to/pqos ..."`
args = []string{"-c", fmt.Sprintf("sudo %s %s", r.PqosPath, strings.ReplaceAll(strings.Join(args, " "), ";", "\\;"))}
cmd = exec.Command("/bin/sh", args...)
}
cmdReader, err := cmd.StdoutPipe()
if err != nil {
r.errorChan <- err
}
go r.processOutput(cmdReader, processesPIDsAssociation)
go func() {
for {
select {
case <-r.stopPQOSChan:
if err := shutDownPqos(cmd); err != nil {
r.Log.Error(err)
}
return
case <-ctx.Done():
if err := shutDownPqos(cmd); err != nil {
r.Log.Error(err)
}
return
}
}
}()
err = cmd.Start()
if err != nil {
r.Log.Errorf("pqos: %v", err)
return
}
err = cmd.Wait()
if err != nil {
r.Log.Errorf("pqos: %v", err)
}
}
func (r *IntelRDT) processOutput(cmdReader io.ReadCloser, processesPIDsAssociation map[string]string) {
reader := bufio.NewScanner(cmdReader)
/*
Omit constant, first 4 lines :
"NOTE: Mixed use of MSR and kernel interfaces to manage
CAT or CMT & MBM may lead to unexpected behavior.\n"
CMT/MBM reset successful
"Time,Core,IPC,LLC Misses,LLC[KB],MBL[MB/s],MBR[MB/s],MBT[MB/s]\n"
*/
toOmit := pqosInitOutputLinesNumber
if len(r.parsedCores) != 0 { // omit first measurements which are zeroes
toOmit = toOmit + len(r.parsedCores)
} else if len(processesPIDsAssociation) != 0 { // specify how many lines should pass before stopping
toOmit = toOmit + len(processesPIDsAssociation)
}
for omitCounter := 0; omitCounter < toOmit; omitCounter++ {
reader.Scan()
}
for reader.Scan() {
out := reader.Text()
// to handle situation when monitored PID disappear and "err" is shown in output
if strings.Contains(out, "err") {
continue
}
if len(r.Processes) != 0 {
newMetric := processMeasurement{}
pids, err := findPIDsInMeasurement(out)
if err != nil {
r.Log.Warnf("Skipping measurement: %v", err)
continue
}
for processName, PIDsProcess := range processesPIDsAssociation {
if pids == PIDsProcess {
newMetric.name = processName
newMetric.measurement = out
}
}
r.publisher.bufferChanProcess <- newMetric
} else {
r.publisher.bufferChanCores <- out
}
}
}
func shutDownPqos(pqos *exec.Cmd) error {
timeout := time.Second * 2
if pqos.Process != nil {
//nolint:errcheck // try to send interrupt signal, ignore err for now
pqos.Process.Signal(os.Interrupt)
// wait and constantly check if pqos is still running
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
for {
if err := pqos.Process.Signal(syscall.Signal(0)); errors.Is(err, os.ErrProcessDone) {
return nil
} else if ctx.Err() != nil {
break
}
}
// if pqos is still running after some period, try to kill it
// this will send SIGTERM to pqos, and leave garbage in `/sys/fs/resctrl/mon_groups`
// fixed in https://github.com/intel/intel-cmt-cat/issues/197
err := pqos.Process.Kill()
if err != nil {
return fmt.Errorf("failed to shut down pqos: %w", err)
}
}
return nil
}
func createArgCores(cores []string) string {
allGroupsArg := "--mon-core="
for _, coreGroup := range cores {
argGroup := createArgsForGroups(strings.Split(coreGroup, ","))
allGroupsArg = allGroupsArg + argGroup
}
return allGroupsArg
}
func createArgProcess(processPIDs map[string]string) string {
allPIDsArg := "--mon-pid="
for _, PIDs := range processPIDs {
argPIDs := createArgsForGroups(strings.Split(PIDs, ","))
allPIDsArg = allPIDsArg + argPIDs
}
return allPIDsArg
}
func createArgsForGroups(coresOrPIDs []string) string {
template := "all:[%s];mbt:[%s];"
group := ""
for _, coreOrPID := range coresOrPIDs {
group = group + coreOrPID + ","
}
if group != "" {
group = strings.TrimSuffix(group, ",")
return fmt.Sprintf(template, group, group)
}
return ""
}
func validatePqosPath(pqosPath string) error {
if len(pqosPath) == 0 {
return errors.New("monitoring start error, can not find pqos executable")
}
pathInfo, err := os.Stat(pqosPath)
if os.IsNotExist(err) {
return errors.New("monitoring start error, provided pqos path not exist")
}
if mode := pathInfo.Mode(); !mode.IsRegular() {
return errors.New("monitoring start error, provided pqos path does not point to a regular file")
}
return nil
}
func parseCoresConfig(cores []string) ([]string, error) {
var allCores []int
parsedCores := make([]string, 0, len(cores))
for _, singleCoreGroup := range cores {
var actualGroupOfCores []int
separatedCores := strings.Split(singleCoreGroup, ",")
for _, coreStr := range separatedCores {
actualCores, err := validateAndParseCores(coreStr)
if err != nil {
return nil, fmt.Errorf("wrong cores input config data format: %w", err)
}
if checkForDuplicates(allCores, actualCores) {
return nil, errors.New("wrong cores input config data format: core value cannot be duplicated")
}
actualGroupOfCores = append(actualGroupOfCores, actualCores...)
allCores = append(allCores, actualGroupOfCores...)
}
parsedCores = append(parsedCores, arrayToString(actualGroupOfCores))
}
return parsedCores, nil
}
func validateAndParseCores(coreStr string) ([]int, error) {
var processedCores []int
if strings.Contains(coreStr, "-") {
rangeValues := strings.Split(coreStr, "-")
if len(rangeValues) != 2 {
return nil, errors.New("more than two values in range")
}
startValue, err := strconv.Atoi(rangeValues[0])
if err != nil {
return nil, err
}
stopValue, err := strconv.Atoi(rangeValues[1])
if err != nil {
return nil, err
}
if startValue > stopValue {
return nil, errors.New("first value cannot be higher than second")
}
rangeOfCores := makeRange(startValue, stopValue)
processedCores = append(processedCores, rangeOfCores...)
} else {
newCore, err := strconv.Atoi(coreStr)
if err != nil {
return nil, err
}
processedCores = append(processedCores, newCore)
}
return processedCores, nil
}
func findPIDsInMeasurement(measurements string) (string, error) {
// to distinguish PIDs from Cores (PIDs should be in quotes)
var insideQuoteRegex = regexp.MustCompile(`"(.*?)"`)
pidsMatch := insideQuoteRegex.FindStringSubmatch(measurements)
if len(pidsMatch) < 2 {
return "", errors.New("cannot find PIDs in measurement line")
}
pids := pidsMatch[1]
return pids, nil
}
func splitCSVLineIntoValues(line string) (splitCSVLine, error) {
values, err := splitMeasurementLine(line)
if err != nil {
return splitCSVLine{}, err
}
timeValue := values[0]
// Because pqos csv format is broken when many cores are involved in PID or
// group of PIDs, there is need to work around it. E.g.:
// Time,PID,Core,IPC,LLC Misses,LLC[KB],MBL[MB/s],MBR[MB/s],MBT[MB/s]
// 2020-08-12 13:34:36,"45417,29170,",37,44,0.00,0,0.0,0.0,0.0,0.0
metricsValues := values[len(values)-numberOfMetrics:]
coreOrPIDsValues := values[1 : len(values)-numberOfMetrics]
return splitCSVLine{timeValue, metricsValues, coreOrPIDsValues}, nil
}
func validateInterval(interval int32) error {
if interval < 0 {
return errors.New("interval cannot be lower than 0")
}
return nil
}
func splitMeasurementLine(line string) ([]string, error) {
values := strings.Split(line, ",")
if len(values) < 8 {
return nil, fmt.Errorf("not valid line format from pqos: %s", values)
}
return values, nil
}
func parseTime(value string) (time.Time, error) {
timestamp, err := time.ParseInLocation(timestampFormat, value, time.Local)
if err != nil {
return time.Time{}, err
}
return timestamp, nil
}
func parseFloat(value string) (float64, error) {
result, err := strconv.ParseFloat(value, 64)
if err != nil {
return result, err
}
return result, nil
}
func arrayToString(array []int) string {
result := ""
for _, value := range array {
result = fmt.Sprintf("%s%d,", result, value)
}
return strings.TrimSuffix(result, ",")
}
func checkForDuplicates(values, valuesToCheck []int) bool {
for _, value := range values {
for _, valueToCheck := range valuesToCheck {
if value == valueToCheck {
return true
}
}
}
return false
}
func makeRange(low, high int) []int {
a := make([]int, high-low+1)
for i := range a {
a[i] = low + i
}
return a
}
func init() {
inputs.Add("intel_rdt", func() telegraf.Input {
rdt := IntelRDT{}
pathPqos, err := exec.LookPath("pqos")
if len(pathPqos) > 0 && err != nil {
rdt.PqosPath = pathPqos
}
return &rdt
})
}

View file

@ -0,0 +1,277 @@
//go:build !windows
package intel_rdt
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/influxdata/telegraf/testutil"
)
type mockProc struct{}
func (*mockProc) getAllProcesses() ([]process, error) {
procs := []process{
{Name: "process", PID: 1000},
{Name: "process2", PID: 1002},
{Name: "process2", PID: 1003},
}
return procs, nil
}
func TestAssociateProcessesWithPIDs(t *testing.T) {
log := testutil.Logger{}
proc := &mockProc{}
rdt := IntelRDT{
Log: log,
processor: proc,
}
processes := []string{"process"}
expectedPID := "1000"
result, err := rdt.associateProcessesWithPIDs(processes)
require.NoError(t, err)
require.Equal(t, expectedPID, result[processes[0]])
processes = []string{"process2"}
expectedPID = "1002,1003"
result, err = rdt.associateProcessesWithPIDs(processes)
require.NoError(t, err)
require.Equal(t, expectedPID, result[processes[0]])
processes = []string{"process1"}
result, err = rdt.associateProcessesWithPIDs(processes)
require.NoError(t, err)
require.Empty(t, result)
}
func TestSplitCSVLineIntoValues(t *testing.T) {
line := "2020-08-12 13:34:36,\"45417,29170\",37,44,0.00,0,0.0,0.0,0.0,0.0"
expectedTimeValue := "2020-08-12 13:34:36"
expectedMetricsValue := []string{"0.00", "0", "0.0", "0.0", "0.0", "0.0"}
expectedCoreOrPidsValue := []string{"\"45417", "29170\"", "37", "44"}
splitCSV, err := splitCSVLineIntoValues(line)
require.NoError(t, err)
require.Equal(t, expectedTimeValue, splitCSV.timeValue)
require.Equal(t, expectedMetricsValue, splitCSV.metricsValues)
require.Equal(t, expectedCoreOrPidsValue, splitCSV.coreOrPIDsValues)
wrongLine := "2020-08-12 13:34:36,37,44,0.00,0,0.0"
splitCSV, err = splitCSVLineIntoValues(wrongLine)
require.Error(t, err)
require.Empty(t, splitCSV.timeValue)
require.Nil(t, splitCSV.metricsValues)
require.Nil(t, splitCSV.coreOrPIDsValues)
}
func TestFindPIDsInMeasurement(t *testing.T) {
line := "2020-08-12 13:34:36,\"45417,29170\""
expected := "45417,29170"
result, err := findPIDsInMeasurement(line)
require.NoError(t, err)
require.Equal(t, expected, result)
line = "pids not included"
result, err = findPIDsInMeasurement(line)
require.Error(t, err)
require.Empty(t, result)
}
func TestCreateArgsProcesses(t *testing.T) {
processesPIDs := map[string]string{
"process": "12345, 99999",
}
expected := "--mon-pid=all:[12345, 99999];mbt:[12345, 99999];"
result := createArgProcess(processesPIDs)
require.EqualValues(t, expected, result)
processesPIDs = map[string]string{
"process": "12345, 99999",
"process2": "44444, 11111",
}
expectedPrefix := "--mon-pid="
expectedSubstring := "all:[12345, 99999];mbt:[12345, 99999];"
expectedSubstring2 := "all:[44444, 11111];mbt:[44444, 11111];"
result = createArgProcess(processesPIDs)
require.Contains(t, result, expectedPrefix)
require.Contains(t, result, expectedSubstring)
require.Contains(t, result, expectedSubstring2)
}
func TestCreateArgsCores(t *testing.T) {
cores := []string{"1,2,3"}
expected := "--mon-core=all:[1,2,3];mbt:[1,2,3];"
result := createArgCores(cores)
require.EqualValues(t, expected, result)
cores = []string{"1,2,3", "4,5,6"}
expectedPrefix := "--mon-core="
expectedSubstring := "all:[1,2,3];mbt:[1,2,3];"
expectedSubstring2 := "all:[4,5,6];mbt:[4,5,6];"
result = createArgCores(cores)
require.Contains(t, result, expectedPrefix)
require.Contains(t, result, expectedSubstring)
require.Contains(t, result, expectedSubstring2)
}
func TestParseCoresConfig(t *testing.T) {
t.Run("empty slice", func(t *testing.T) {
var configCores []string
result, err := parseCoresConfig(configCores)
require.NoError(t, err)
require.Empty(t, result)
})
t.Run("empty string in slice", func(t *testing.T) {
configCores := []string{""}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Empty(t, result)
})
t.Run("not correct string", func(t *testing.T) {
configCores := []string{"wrong string"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Empty(t, result)
})
t.Run("not correct string", func(t *testing.T) {
configCores := []string{"1,2", "wasd:#$!;"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Empty(t, result)
})
t.Run("not correct string", func(t *testing.T) {
configCores := []string{"1,2,2"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Empty(t, result)
})
t.Run("coma separated cores - positive", func(t *testing.T) {
configCores := []string{"0,1,2,3,4,5"}
expected := []string{"0,1,2,3,4,5"}
result, err := parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"0,1,2", "3,4,5"}
expected = []string{"0,1,2", "3,4,5"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"0,4,1", "2,3,5", "9"}
expected = []string{"0,4,1", "2,3,5", "9"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
})
t.Run("coma separated cores - negative", func(t *testing.T) {
// cannot monitor same cores in different groups
configCores := []string{"0,1,2", "2"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
configCores = []string{"0,1,2", "2,3,4"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
configCores = []string{"0,-1,2", "2,3,4"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
})
t.Run("dash separated cores - positive", func(t *testing.T) {
configCores := []string{"0-5"}
expected := []string{"0,1,2,3,4,5"}
result, err := parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"0-5", "7-10"}
expected = []string{"0,1,2,3,4,5", "7,8,9,10"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"5-5"}
expected = []string{"5"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
})
t.Run("dash separated cores - negative", func(t *testing.T) {
// cannot monitor same cores in different groups
configCores := []string{"0-5", "2-7"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
// more than two values in range
configCores = []string{"0-5-10"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
// first value cannot be higher than second
configCores = []string{"12-5"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
configCores = []string{"0-"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
})
t.Run("mixed separator - positive", func(t *testing.T) {
configCores := []string{"0-5,6,7"}
expected := []string{"0,1,2,3,4,5,6,7"}
result, err := parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"0-5,6,7", "8,9,10"}
expected = []string{"0,1,2,3,4,5,6,7", "8,9,10"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
configCores = []string{"0-7", "8-10"}
expected = []string{"0,1,2,3,4,5,6,7", "8,9,10"}
result, err = parseCoresConfig(configCores)
require.NoError(t, err)
require.EqualValues(t, expected, result)
})
t.Run("mixed separator - negative", func(t *testing.T) {
// cannot monitor same cores in different groups
configCores := []string{"0-5,", "2-7"}
result, err := parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
// cores cannot be duplicated
configCores = []string{"0-5,5"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
// more than two values in range
configCores = []string{"0-5-6,9"}
result, err = parseCoresConfig(configCores)
require.Error(t, err)
require.Nil(t, result)
})
}

View file

@ -0,0 +1,36 @@
//go:build windows
package intel_rdt
import (
_ "embed"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/plugins/inputs"
)
//go:embed sample.conf
var sampleConfig string
type IntelRDT struct {
Log telegraf.Logger `toml:"-"`
}
func (*IntelRDT) SampleConfig() string { return sampleConfig }
func (i *IntelRDT) Init() error {
i.Log.Warn("Current platform is not supported")
return nil
}
func (*IntelRDT) Start(_ telegraf.Accumulator) error { return nil }
func (*IntelRDT) Gather(_ telegraf.Accumulator) error { return nil }
func (*IntelRDT) Stop() {}
func init() {
inputs.Add("intel_rdt", func() telegraf.Input {
return &IntelRDT{}
})
}

View file

@ -0,0 +1,41 @@
//go:build !windows
package intel_rdt
import "github.com/prometheus/procfs"
type processesHandler interface {
getAllProcesses() ([]process, error)
}
type process struct {
Name string
PID int
}
type processManager struct{}
func newProcessor() processesHandler {
return &processManager{}
}
func (*processManager) getAllProcesses() ([]process, error) {
allProcesses, err := procfs.AllProcs()
if err != nil {
return nil, err
}
processes := make([]process, 0, len(allProcesses))
for _, proc := range allProcesses {
procComm, err := proc.Comm()
if err != nil {
continue
}
newProcess := process{
PID: proc.PID,
Name: procComm,
}
processes = append(processes, newProcess)
}
return processes, nil
}

View file

@ -0,0 +1,176 @@
//go:build !windows
package intel_rdt
import (
"context"
"errors"
"strings"
"time"
"github.com/influxdata/telegraf"
)
type parsedCoresMeasurement struct {
cores string
values []float64
time time.Time
}
type parsedProcessMeasurement struct {
process string
cores string
values []float64
time time.Time
}
// publisher for publish new RDT metrics to telegraf accumulator
type publisher struct {
acc telegraf.Accumulator
log telegraf.Logger
shortenedMetrics bool
bufferChanProcess chan processMeasurement
bufferChanCores chan string
errChan chan error
}
func newPublisher(acc telegraf.Accumulator, log telegraf.Logger, shortenedMetrics bool) publisher {
return publisher{
acc: acc,
log: log,
shortenedMetrics: shortenedMetrics,
bufferChanProcess: make(chan processMeasurement),
bufferChanCores: make(chan string),
errChan: make(chan error),
}
}
func (p *publisher) publish(ctx context.Context) {
go func() {
for {
select {
case newMeasurements := <-p.bufferChanCores:
p.publishCores(newMeasurements)
case newMeasurements := <-p.bufferChanProcess:
p.publishProcess(newMeasurements)
case err := <-p.errChan:
p.log.Error(err)
case <-ctx.Done():
return
}
}
}()
}
func (p *publisher) publishCores(measurement string) {
parsedCoresMeasurement, err := parseCoresMeasurement(measurement)
if err != nil {
p.errChan <- err
}
p.addToAccumulatorCores(parsedCoresMeasurement)
}
func (p *publisher) publishProcess(measurement processMeasurement) {
parsedProcessMeasurement, err := parseProcessesMeasurement(measurement)
if err != nil {
p.errChan <- err
}
p.addToAccumulatorProcesses(parsedProcessMeasurement)
}
func parseCoresMeasurement(measurements string) (parsedCoresMeasurement, error) {
splitCSV, err := splitCSVLineIntoValues(measurements)
if err != nil {
return parsedCoresMeasurement{}, err
}
timestamp, err := parseTime(splitCSV.timeValue)
if err != nil {
return parsedCoresMeasurement{}, err
}
// change string slice to one string and separate it by coma
coresString := strings.Join(splitCSV.coreOrPIDsValues, ",")
// trim unwanted quotes
coresString = strings.Trim(coresString, "\"")
values := make([]float64, 0, len(splitCSV.metricsValues))
for _, metric := range splitCSV.metricsValues {
parsedValue, err := parseFloat(metric)
if err != nil {
return parsedCoresMeasurement{}, err
}
values = append(values, parsedValue)
}
return parsedCoresMeasurement{coresString, values, timestamp}, nil
}
func (p *publisher) addToAccumulatorCores(measurement parsedCoresMeasurement) {
for i, value := range measurement.values {
if p.shortenedMetrics {
// 0: "IPC"
// 1: "LLC_Misses"
if i == 0 || i == 1 {
continue
}
}
tags := make(map[string]string, 2)
fields := make(map[string]interface{}, 1)
tags["cores"] = measurement.cores
tags["name"] = pqosMetricOrder[i]
fields["value"] = value
p.acc.AddFields("rdt_metric", fields, tags, measurement.time)
}
}
func parseProcessesMeasurement(measurement processMeasurement) (parsedProcessMeasurement, error) {
splitCSV, err := splitCSVLineIntoValues(measurement.measurement)
if err != nil {
return parsedProcessMeasurement{}, err
}
pids, err := findPIDsInMeasurement(measurement.measurement)
if err != nil {
return parsedProcessMeasurement{}, err
}
lenOfPIDs := len(strings.Split(pids, ","))
if lenOfPIDs > len(splitCSV.coreOrPIDsValues) {
return parsedProcessMeasurement{}, errors.New("detected more pids (quoted) than actual number of pids in csv line")
}
timestamp, err := parseTime(splitCSV.timeValue)
if err != nil {
return parsedProcessMeasurement{}, err
}
actualProcess := measurement.name
cores := strings.Trim(strings.Join(splitCSV.coreOrPIDsValues[lenOfPIDs:], ","), `"`)
values := make([]float64, 0, len(splitCSV.metricsValues))
for _, metric := range splitCSV.metricsValues {
parsedValue, err := parseFloat(metric)
if err != nil {
return parsedProcessMeasurement{}, err
}
values = append(values, parsedValue)
}
return parsedProcessMeasurement{actualProcess, cores, values, timestamp}, nil
}
func (p *publisher) addToAccumulatorProcesses(measurement parsedProcessMeasurement) {
for i, value := range measurement.values {
if p.shortenedMetrics {
// 0: "IPC"
// 1: "LLC_Misses"
if i == 0 || i == 1 {
continue
}
}
tags := make(map[string]string, 3)
fields := make(map[string]interface{}, 1)
tags["process"] = measurement.process
tags["cores"] = measurement.cores
tags["name"] = pqosMetricOrder[i]
fields["value"] = value
p.acc.AddFields("rdt_metric", fields, tags, measurement.time)
}
}

View file

@ -0,0 +1,435 @@
//go:build !windows
package intel_rdt
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/influxdata/telegraf/testutil"
)
var metricsValues = map[string]float64{
"IPC": 0.5,
"LLC_Misses": 61650,
"LLC": 1632,
"MBL": 0.6,
"MBR": 0.9,
"MBT": 1.9,
}
func TestParseCoresMeasurement(t *testing.T) {
timestamp := "2020-08-12 13:34:36"
cores := "\"37,44\""
t.Run("valid measurement string", func(t *testing.T) {
measurement := fmt.Sprintf("%s,%s,%f,%f,%f,%f,%f,%f",
timestamp,
cores,
metricsValues["IPC"],
metricsValues["LLC_Misses"],
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"])
expectedCores := "37,44"
expectedTimestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
result, err := parseCoresMeasurement(measurement)
require.NoError(t, err)
require.Equal(t, expectedCores, result.cores)
require.Equal(t, expectedTimestamp, result.time)
require.InDelta(t, result.values[0], metricsValues["IPC"], testutil.DefaultDelta)
require.InDelta(t, result.values[1], metricsValues["LLC_Misses"], testutil.DefaultDelta)
require.InDelta(t, result.values[2], metricsValues["LLC"], testutil.DefaultDelta)
require.InDelta(t, result.values[3], metricsValues["MBL"], testutil.DefaultDelta)
require.InDelta(t, result.values[4], metricsValues["MBR"], testutil.DefaultDelta)
require.InDelta(t, result.values[5], metricsValues["MBT"], testutil.DefaultDelta)
})
t.Run("not valid measurement string", func(t *testing.T) {
measurement := "not, valid, measurement"
result, err := parseCoresMeasurement(measurement)
require.Error(t, err)
require.Empty(t, result.cores)
require.Nil(t, result.values)
require.Equal(t, time.Time{}, result.time)
})
t.Run("not valid values string", func(t *testing.T) {
measurement := fmt.Sprintf("%s,%s,%s,%s,%f,%f,%f,%f",
timestamp,
cores,
"%d",
"in",
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"])
result, err := parseCoresMeasurement(measurement)
require.Error(t, err)
require.Empty(t, result.cores)
require.Nil(t, result.values)
require.Equal(t, time.Time{}, result.time)
})
t.Run("not valid timestamp format", func(t *testing.T) {
invalidTimestamp := "2020-08-12-21 13:34:"
measurement := fmt.Sprintf("%s,%s,%f,%f,%f,%f,%f,%f",
invalidTimestamp,
cores,
metricsValues["IPC"],
metricsValues["LLC_Misses"],
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"])
result, err := parseCoresMeasurement(measurement)
require.Error(t, err)
require.Empty(t, result.cores)
require.Nil(t, result.values)
require.Equal(t, time.Time{}, result.time)
})
}
func TestParseProcessesMeasurement(t *testing.T) {
timestamp := "2020-08-12 13:34:36"
cores := "\"37,44\""
pids := "\"12345,9999\""
processName := "process_name"
t.Run("valid measurement string", func(t *testing.T) {
measurement := fmt.Sprintf("%s,%s,%s,%f,%f,%f,%f,%f,%f",
timestamp,
pids,
cores,
metricsValues["IPC"],
metricsValues["LLC_Misses"],
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"])
expectedCores := "37,44"
expectedTimestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
newMeasurement := processMeasurement{
name: processName,
measurement: measurement,
}
result, err := parseProcessesMeasurement(newMeasurement)
require.NoError(t, err)
require.Equal(t, processName, result.process)
require.Equal(t, expectedCores, result.cores)
require.Equal(t, expectedTimestamp, result.time)
require.InDelta(t, result.values[0], metricsValues["IPC"], testutil.DefaultDelta)
require.InDelta(t, result.values[1], metricsValues["LLC_Misses"], testutil.DefaultDelta)
require.InDelta(t, result.values[2], metricsValues["LLC"], testutil.DefaultDelta)
require.InDelta(t, result.values[3], metricsValues["MBL"], testutil.DefaultDelta)
require.InDelta(t, result.values[4], metricsValues["MBR"], testutil.DefaultDelta)
require.InDelta(t, result.values[5], metricsValues["MBT"], testutil.DefaultDelta)
})
invalidTimestamp := "2020-20-20-31"
negativeTests := []struct {
name string
measurement string
}{{
name: "not valid measurement string",
measurement: "invalid,measurement,format",
}, {
name: "not valid timestamp format",
measurement: fmt.Sprintf("%s,%s,%s,%f,%f,%f,%f,%f,%f",
invalidTimestamp,
pids,
cores,
metricsValues["IPC"],
metricsValues["LLC_Misses"],
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"]),
},
{
name: "not valid values string",
measurement: fmt.Sprintf("%s,%s,%s,%s,%s,%f,%f,%f,%f",
timestamp,
pids,
cores,
"1##",
"da",
metricsValues["LLC"],
metricsValues["MBL"],
metricsValues["MBR"],
metricsValues["MBT"]),
},
{
name: "not valid csv line with quotes",
measurement: "0000-08-02 0:00:00,,\",,,,,,,,,,,,,,,,,,,,,,,,\",,",
},
}
for _, test := range negativeTests {
t.Run(test.name, func(t *testing.T) {
newMeasurement := processMeasurement{
name: processName,
measurement: test.measurement,
}
result, err := parseProcessesMeasurement(newMeasurement)
require.Error(t, err)
require.Empty(t, result.process)
require.Empty(t, result.cores)
require.Nil(t, result.values)
require.Equal(t, time.Time{}, result.time)
})
}
}
func TestAddToAccumulatorCores(t *testing.T) {
t.Run("shortened false", func(t *testing.T) {
var acc testutil.Accumulator
publisher := publisher{acc: &acc}
cores := "1,2,3"
metricsValues := []float64{1, 2, 3, 4, 5, 6}
timestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
publisher.addToAccumulatorCores(parsedCoresMeasurement{cores, metricsValues, timestamp})
for _, test := range testCoreMetrics {
acc.AssertContainsTaggedFields(t, "rdt_metric", test.fields, test.tags)
}
})
t.Run("shortened true", func(t *testing.T) {
var acc testutil.Accumulator
publisher := publisher{acc: &acc, shortenedMetrics: true}
cores := "1,2,3"
metricsValues := []float64{1, 2, 3, 4, 5, 6}
timestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
publisher.addToAccumulatorCores(parsedCoresMeasurement{cores, metricsValues, timestamp})
for _, test := range testCoreMetricsShortened {
acc.AssertDoesNotContainsTaggedFields(t, "rdt_metric", test.fields, test.tags)
}
})
}
func TestAddToAccumulatorProcesses(t *testing.T) {
t.Run("shortened false", func(t *testing.T) {
var acc testutil.Accumulator
publisher := publisher{acc: &acc}
process := "process_name"
cores := "1,2,3"
metricsValues := []float64{1, 2, 3, 4, 5, 6}
timestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
publisher.addToAccumulatorProcesses(parsedProcessMeasurement{process, cores, metricsValues, timestamp})
for _, test := range testCoreProcesses {
acc.AssertContainsTaggedFields(t, "rdt_metric", test.fields, test.tags)
}
})
t.Run("shortened true", func(t *testing.T) {
var acc testutil.Accumulator
publisher := publisher{acc: &acc, shortenedMetrics: true}
process := "process_name"
cores := "1,2,3"
metricsValues := []float64{1, 2, 3, 4, 5, 6}
timestamp := time.Date(2020, 8, 12, 13, 34, 36, 0, time.Local)
publisher.addToAccumulatorProcesses(parsedProcessMeasurement{process, cores, metricsValues, timestamp})
for _, test := range testCoreProcessesShortened {
acc.AssertDoesNotContainsTaggedFields(t, "rdt_metric", test.fields, test.tags)
}
})
}
var (
testCoreMetrics = []struct {
fields map[string]interface{}
tags map[string]string
}{
{
map[string]interface{}{
"value": float64(1),
},
map[string]string{
"cores": "1,2,3",
"name": "IPC",
},
},
{
map[string]interface{}{
"value": float64(2),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC_Misses",
},
},
{
map[string]interface{}{
"value": float64(3),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC",
},
},
{
map[string]interface{}{
"value": float64(4),
},
map[string]string{
"cores": "1,2,3",
"name": "MBL",
},
},
{
map[string]interface{}{
"value": float64(5),
},
map[string]string{
"cores": "1,2,3",
"name": "MBR",
},
},
{
map[string]interface{}{
"value": float64(6),
},
map[string]string{
"cores": "1,2,3",
"name": "MBT",
},
},
}
testCoreMetricsShortened = []struct {
fields map[string]interface{}
tags map[string]string
}{
{
map[string]interface{}{
"value": float64(1),
},
map[string]string{
"cores": "1,2,3",
"name": "IPC",
},
},
{
map[string]interface{}{
"value": float64(2),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC_Misses",
},
},
}
testCoreProcesses = []struct {
fields map[string]interface{}
tags map[string]string
}{
{
map[string]interface{}{
"value": float64(1),
},
map[string]string{
"cores": "1,2,3",
"name": "IPC",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(2),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC_Misses",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(3),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(4),
},
map[string]string{
"cores": "1,2,3",
"name": "MBL",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(5),
},
map[string]string{
"cores": "1,2,3",
"name": "MBR",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(6),
},
map[string]string{
"cores": "1,2,3",
"name": "MBT",
"process": "process_name",
},
},
}
testCoreProcessesShortened = []struct {
fields map[string]interface{}
tags map[string]string
}{
{
map[string]interface{}{
"value": float64(1),
},
map[string]string{
"cores": "1,2,3",
"name": "IPC",
"process": "process_name",
},
},
{
map[string]interface{}{
"value": float64(2),
},
map[string]string{
"cores": "1,2,3",
"name": "LLC_Misses",
"process": "process_name",
},
},
}
)

View file

@ -0,0 +1,29 @@
# Read Intel RDT metrics
# This plugin ONLY supports non-Windows
[[inputs.intel_rdt]]
## Optionally set sampling interval to Nx100ms.
## This value is propagated to pqos tool. Interval format is defined by pqos itself.
## If not provided or provided 0, will be set to 10 = 10x100ms = 1s.
# sampling_interval = "10"
## Optionally specify the path to pqos executable.
## If not provided, auto discovery will be performed.
# pqos_path = "/usr/local/bin/pqos"
## Optionally specify if IPC and LLC_Misses metrics shouldn't be propagated.
## If not provided, default value is false.
# shortened_metrics = false
## Specify the list of groups of CPU core(s) to be provided as pqos input.
## Mandatory if processes aren't set and forbidden if processes are specified.
## e.g. ["0-3", "4,5,6"] or ["1-3,4"]
# cores = ["0-3"]
## Specify the list of processes for which Metrics will be collected.
## Mandatory if cores aren't set and forbidden if cores are specified.
## e.g. ["qemu", "pmd"]
# processes = ["process"]
## Specify if the pqos process should be called with sudo.
## Mandatory if the telegraf process does not run as root.
# use_sudo = false