Add cpu/cpuset cgroup and procfs data gathering (#11763)

* cpu/cpuset cgroup and procfs data gathering

* Renames and default values

* Formatting

* Trigger Build

* Add cgroup monitors

* Return 0 if no period

* Update

Co-authored-by: arunramani-imply <84351090+arunramani-imply@users.noreply.github.com>
This commit is contained in:
Arun Ramani 2021-10-06 20:27:36 -07:00 committed by GitHub
parent 8fd17fe0af
commit 15789137a3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 1179 additions and 1 deletions

View File

@ -0,0 +1,76 @@
/*
* 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.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.Cpu;
import java.util.Map;
public class CgroupCpuMonitor extends FeedDefiningMonitor
{
final CgroupDiscoverer cgroupDiscoverer;
final Map<String, String[]> dimensions;
public CgroupCpuMonitor(CgroupDiscoverer cgroupDiscoverer, final Map<String, String[]> dimensions, String feed)
{
super(feed);
this.cgroupDiscoverer = cgroupDiscoverer;
this.dimensions = dimensions;
}
public CgroupCpuMonitor(final Map<String, String[]> dimensions, String feed)
{
this(null, dimensions, feed);
}
public CgroupCpuMonitor(final Map<String, String[]> dimensions)
{
this(dimensions, DEFAULT_METRICS_FEED);
}
public CgroupCpuMonitor()
{
this(ImmutableMap.of());
}
@Override
public boolean doMonitor(ServiceEmitter emitter)
{
final Cpu cpu = new Cpu(cgroupDiscoverer);
final Cpu.CpuAllocationMetric cpuSnapshot = cpu.snapshot();
final ServiceMetricEvent.Builder builder = builder();
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
emitter.emit(builder.build("cgroup/cpu/shares", cpuSnapshot.getShares()));
emitter.emit(builder.build(
"cgroup/cpu/cores_quota",
cpuSnapshot.getPeriodUs() == 0
? 0
: ((double) cpuSnapshot.getQuotaUs()
) / cpuSnapshot.getPeriodUs()
));
return true;
}
}

View File

@ -0,0 +1,83 @@
/*
* 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.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.CpuSet;
import java.util.Map;
public class CgroupCpuSetMonitor extends FeedDefiningMonitor
{
final CgroupDiscoverer cgroupDiscoverer;
final Map<String, String[]> dimensions;
public CgroupCpuSetMonitor(CgroupDiscoverer cgroupDiscoverer, final Map<String, String[]> dimensions, String feed)
{
super(feed);
this.cgroupDiscoverer = cgroupDiscoverer;
this.dimensions = dimensions;
}
public CgroupCpuSetMonitor(final Map<String, String[]> dimensions, String feed)
{
this(null, dimensions, feed);
}
public CgroupCpuSetMonitor(final Map<String, String[]> dimensions)
{
this(dimensions, DEFAULT_METRICS_FEED);
}
public CgroupCpuSetMonitor()
{
this(ImmutableMap.of());
}
@Override
public boolean doMonitor(ServiceEmitter emitter)
{
final CpuSet cpuset = new CpuSet(cgroupDiscoverer);
final CpuSet.CpuSetMetric cpusetSnapshot = cpuset.snapshot();
final ServiceMetricEvent.Builder builder = builder();
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
emitter.emit(builder.build(
"cgroup/cpuset/cpu_count",
cpusetSnapshot.getCpuSetCpus().length
));
emitter.emit(builder.build(
"cgroup/cpuset/effective_cpu_count",
cpusetSnapshot.getEffectiveCpuSetCpus().length
));
emitter.emit(builder.build(
"cgroup/cpuset/mems_count",
cpusetSnapshot.getCpuSetMems().length
));
emitter.emit(builder.build(
"cgroup/cpuset/effective_mems_count",
cpusetSnapshot.getEffectiveCpuSetMems().length
));
return true;
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.druid.java.util.metrics;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.logger.Logger;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.List;
import java.util.UUID;
/**
* Fetches data from top-level procfs files for metrics.
*/
public class ProcFsReader
{
private final File procDir;
private static final Logger LOG = new Logger(ProcFsReader.class);
public static final Path DEFAULT_PROC_FS_ROOT = Paths.get("/proc/");
private static final String BOOT_ID_PATH = "sys/kernel/random/boot_id";
private static final String CPUINFO_PATH = "cpuinfo";
public ProcFsReader(Path procFsRoot)
{
this.procDir = Preconditions.checkNotNull(procFsRoot, "procFsRoot").toFile();
Preconditions.checkArgument(this.procDir.isDirectory(), "Not a directory: [%s]", procFsRoot);
}
/**
* Reads the boot ID from the boot_id path, which is just a UUID.
*
* @return The boot UUID.
*/
public UUID getBootId()
{
Path path = Paths.get(this.procDir.toString(), BOOT_ID_PATH);
try {
List<String> lines = Files.readAllLines(path);
return lines.stream().map(UUID::fromString).findFirst().orElse(new UUID(0, 0));
}
catch (IOException ex) {
LOG.error(ex, "Unable to read %s", path);
return new UUID(0, 0);
}
}
/**
* Reads the cpuinfo path (example in src/test/resources/cpuinfo) and
* counts the number of processors.
*
* @return the number of processors.
*/
public long getProcessorCount()
{
Path path = Paths.get(this.procDir.toString(), CPUINFO_PATH);
try {
List<String> lines = Files.readAllLines(path);
return lines.stream().filter(l -> l.startsWith("processor")).count();
}
catch (IOException ex) {
LOG.error(ex, "Unable to read %s", path);
return -1L;
}
}
}

View File

@ -0,0 +1,110 @@
/*
* 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.druid.java.util.metrics.cgroups;
import com.google.common.primitives.Longs;
import org.apache.druid.java.util.common.logger.Logger;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.List;
import java.util.Objects;
/**
* Collect CPU share and quota information from cpu cgroup files.
*/
public class Cpu
{
private static final Logger LOG = new Logger(Cpu.class);
private static final String CGROUP = "cpu";
private static final String CPU_SHARES_FILE = "cpu.shares";
private static final String CPU_QUOTA_FILE = "cpu.cfs_quota_us";
private static final String CPU_PERIOD_FILE = "cpu.cfs_period_us";
private final CgroupDiscoverer cgroupDiscoverer;
public Cpu(CgroupDiscoverer cgroupDiscoverer)
{
this.cgroupDiscoverer = cgroupDiscoverer;
}
/**
* Take a snapshot of cpu cgroup data
*
* @return A snapshot with the data populated.
*/
public CpuAllocationMetric snapshot()
{
return new CpuAllocationMetric(
readLongValue(CPU_SHARES_FILE, -1),
readLongValue(CPU_QUOTA_FILE, 0),
readLongValue(CPU_PERIOD_FILE, 0)
);
}
private long readLongValue(String fileName, long defaultValeue)
{
try {
List<String> lines = Files.readAllLines(Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), fileName));
return lines.stream().map(Longs::tryParse).filter(Objects::nonNull).findFirst().orElse(defaultValeue);
}
catch (RuntimeException | IOException ex) {
LOG.error(ex, "Unable to fetch %s", fileName);
return defaultValeue;
}
}
public static class CpuAllocationMetric
{
// Maps to cpu.shares - the share of CPU given to the process
private final long shares;
// Maps to cpu.cfs_quota_us - the maximum time in microseconds during each cfs_period_us
// in for the current group will be allowed to run
private final long quotaUs;
// Maps to cpu.cfs_period_us - the duration in microseconds of each scheduler period, for
// bandwidth decisions
private final long periodUs;
CpuAllocationMetric(long shares, long quotaUs, long periodUs)
{
this.shares = shares;
this.quotaUs = quotaUs;
this.periodUs = periodUs;
}
public final long getShares()
{
return shares;
}
public final long getQuotaUs()
{
return quotaUs;
}
public final long getPeriodUs()
{
return periodUs;
}
}
}

View File

@ -0,0 +1,161 @@
/*
* 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.druid.java.util.metrics.cgroups;
import com.google.common.primitives.Ints;
import org.apache.druid.java.util.common.logger.Logger;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.List;
import java.util.stream.IntStream;
/**
* Collect CPU and memory data from cpuset cgroup files.
*/
public class CpuSet
{
private static final Logger LOG = new Logger(CpuSet.class);
private static final String CGROUP = "cpuset";
private static final String CPUS_FILE = "cpuset.cpus";
private static final String EFFECTIVE_CPUS_FILE = "cpuset.effective_cpus";
private static final String MEMS_FILE = "cpuset.mems";
private static final String EFFECTIVE_MEMS_FILE = "cpuset.effective_mems";
private final CgroupDiscoverer cgroupDiscoverer;
public CpuSet(CgroupDiscoverer cgroupDiscoverer)
{
this.cgroupDiscoverer = cgroupDiscoverer;
}
/**
* Take a snapshot of cpuset cgroup data
*
* @return A snapshot with the data populated.
*/
public CpuSetMetric snapshot()
{
return new CpuSetMetric(
readCpuSetFile(CPUS_FILE),
readCpuSetFile(EFFECTIVE_CPUS_FILE),
readCpuSetFile(MEMS_FILE),
readCpuSetFile(EFFECTIVE_MEMS_FILE)
);
}
private int[] readCpuSetFile(String file)
{
int[] output = {};
try {
List<String> lines = Files.readAllLines(
Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), file));
output = lines.stream().map(this::parseStringRangeToIntArray).findFirst().orElse(output);
}
catch (RuntimeException | IOException ex) {
LOG.error(ex, "Unable to read %s", file);
}
return output;
}
/**
* Parses the cpuset list format and outputs it as a list of CPUs. Examples:
* 0-4,9 # bits 0, 1, 2, 3, 4, and 9 set
* # outputs [0, 1, 2, 3, 4, 9]
* 0-2,7,12-14 # bits 0, 1, 2, 7, 12, 13, and 14 set
* # outputs [0, 1, 2, 7, 12, 13, 14]
*
* This method also works fine for memory nodes.
*
* @param line The list format cpu value
* @return the list of CPU IDs
*/
private int[] parseStringRangeToIntArray(String line)
{
String[] cpuParts = line.split(",");
return Arrays.stream(cpuParts)
.flatMapToInt(cpuPart -> {
String[] bits = cpuPart.split("-");
if (bits.length == 2) {
Integer low = Ints.tryParse(bits[0]);
Integer high = Ints.tryParse(bits[1]);
if (low != null && high != null) {
return IntStream.rangeClosed(low, high);
}
} else if (bits.length == 1) {
Integer bit = Ints.tryParse(bits[0]);
if (bit != null) {
return IntStream.of(bit);
}
}
return IntStream.empty();
}).toArray();
}
public static class CpuSetMetric
{
// The list of processor IDs associated with the process
private final int[] cpuSetCpus;
// The list of effective/active processor IDs associated with the process
private final int[] effectiveCpuSetCpus;
// The list memory nodes associated with the process
private final int[] cpuSetMems;
// The list of effective/active memory nodes associated with the process
private final int[] effectiveCpuSetMems;
CpuSetMetric(
int[] cpuSetCpus,
int[] effectiveCpuSetCpus,
int[] cpuSetMems,
int[] effectiveCpuSetMems)
{
this.cpuSetCpus = cpuSetCpus;
this.effectiveCpuSetCpus = effectiveCpuSetCpus;
this.cpuSetMems = cpuSetMems;
this.effectiveCpuSetMems = effectiveCpuSetMems;
}
public int[] getCpuSetCpus()
{
return cpuSetCpus;
}
public int[] getEffectiveCpuSetCpus()
{
return effectiveCpuSetCpus;
}
public int[] getCpuSetMems()
{
return cpuSetMems;
}
public int[] getEffectiveCpuSetMems()
{
return effectiveCpuSetMems;
}
}
}

View File

@ -0,0 +1,82 @@
/*
* 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.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcCgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.TestUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class CgroupCpuMonitorTest
{
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private File procDir;
private File cgroupDir;
private CgroupDiscoverer discoverer;
@Before
public void setUp() throws IOException
{
cgroupDir = temporaryFolder.newFolder();
procDir = temporaryFolder.newFolder();
discoverer = new ProcCgroupDiscoverer(procDir.toPath());
TestUtils.setUpCgroups(procDir, cgroupDir);
final File cpuDir = new File(
cgroupDir,
"cpu,cpuacct/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
);
Assert.assertTrue((cpuDir.isDirectory() && cpuDir.exists()) || cpuDir.mkdirs());
TestUtils.copyOrReplaceResource("/cpu.shares", new File(cpuDir, "cpu.shares"));
TestUtils.copyOrReplaceResource("/cpu.cfs_quota_us", new File(cpuDir, "cpu.cfs_quota_us"));
TestUtils.copyOrReplaceResource("/cpu.cfs_period_us", new File(cpuDir, "cpu.cfs_period_us"));
}
@Test
public void testMonitor()
{
final CgroupCpuMonitor monitor = new CgroupCpuMonitor(discoverer, ImmutableMap.of(), "some_feed");
final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
Assert.assertTrue(monitor.doMonitor(emitter));
final List<Event> actualEvents = emitter.getEvents();
Assert.assertEquals(2, actualEvents.size());
final Map<String, Object> sharesEvent = actualEvents.get(0).toMap();
final Map<String, Object> coresEvent = actualEvents.get(1).toMap();
Assert.assertEquals("cgroup/cpu/shares", sharesEvent.get("metric"));
Assert.assertEquals(1024L, sharesEvent.get("value"));
Assert.assertEquals("cgroup/cpu/cores_quota", coresEvent.get("metric"));
Assert.assertEquals(3.0D, coresEvent.get("value"));
}
}

View File

@ -0,0 +1,89 @@
/*
* 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.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcCgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.TestUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class CgroupCpuSetMonitorTest
{
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private File procDir;
private File cgroupDir;
private CgroupDiscoverer discoverer;
@Before
public void setUp() throws IOException
{
cgroupDir = temporaryFolder.newFolder();
procDir = temporaryFolder.newFolder();
discoverer = new ProcCgroupDiscoverer(procDir.toPath());
TestUtils.setUpCgroups(procDir, cgroupDir);
final File cpusetDir = new File(
cgroupDir,
"cpuset/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
);
Assert.assertTrue((cpusetDir.isDirectory() && cpusetDir.exists()) || cpusetDir.mkdirs());
TestUtils.copyOrReplaceResource("/cpuset.cpus", new File(cpusetDir, "cpuset.cpus"));
TestUtils.copyOrReplaceResource("/cpuset.effective_cpus.complex", new File(cpusetDir, "cpuset.effective_cpus"));
TestUtils.copyOrReplaceResource("/cpuset.mems", new File(cpusetDir, "cpuset.mems"));
TestUtils.copyOrReplaceResource("/cpuset.effective_mems", new File(cpusetDir, "cpuset.effective_mems"));
}
@Test
public void testMonitor()
{
final CgroupCpuSetMonitor monitor = new CgroupCpuSetMonitor(discoverer, ImmutableMap.of(), "some_feed");
final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
Assert.assertTrue(monitor.doMonitor(emitter));
final List<Event> actualEvents = emitter.getEvents();
Assert.assertEquals(4, actualEvents.size());
final Map<String, Object> cpusEvent = actualEvents.get(0).toMap();
final Map<String, Object> effectiveCpusEvent = actualEvents.get(1).toMap();
final Map<String, Object> memsEvent = actualEvents.get(2).toMap();
final Map<String, Object> effectiveMemsEvent = actualEvents.get(3).toMap();
Assert.assertEquals("cgroup/cpuset/cpu_count", cpusEvent.get("metric"));
Assert.assertEquals(8, cpusEvent.get("value"));
Assert.assertEquals("cgroup/cpuset/effective_cpu_count", effectiveCpusEvent.get("metric"));
Assert.assertEquals(7, effectiveCpusEvent.get("value"));
Assert.assertEquals("cgroup/cpuset/mems_count", memsEvent.get("metric"));
Assert.assertEquals(4, memsEvent.get("value"));
Assert.assertEquals("cgroup/cpuset/effective_mems_count", effectiveMemsEvent.get("metric"));
Assert.assertEquals(1, effectiveMemsEvent.get("value"));
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.druid.java.util.metrics;
import org.apache.druid.java.util.metrics.cgroups.TestUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.nio.file.Paths;
public class ProcFsReaderTest
{
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private File procDir;
@Before
public void setUp() throws IOException
{
procDir = temporaryFolder.newFolder();
File kernelDir = new File(
procDir,
"sys/kernel/random"
);
Assert.assertTrue(kernelDir.mkdirs());
TestUtils.copyResource("/cpuinfo", new File(procDir, "cpuinfo"));
TestUtils.copyResource("/boot_id", new File(kernelDir, "boot_id"));
}
@Test
public void testUtilThrowsOnBadDir()
{
Assert.assertThrows(
IllegalArgumentException.class,
() -> new ProcFsReader(Paths.get(procDir + "_dummy"))
);
}
@Test
public void testBootId()
{
final ProcFsReader fetcher = new ProcFsReader(procDir.toPath());
Assert.assertEquals("ad1f0a5c-55ea-4a49-9db8-bbb0f22e2ba6", fetcher.getBootId().toString());
}
@Test
public void testProcessorCount()
{
final ProcFsReader fetcher = new ProcFsReader(procDir.toPath());
Assert.assertEquals(8, fetcher.getProcessorCount());
}
}

View File

@ -0,0 +1,96 @@
/*
* 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.druid.java.util.metrics.cgroups;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
public class CpuSetTest
{
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private CgroupDiscoverer discoverer;
private File cpusetDir;
@Before
public void setUp() throws IOException
{
File cgroupDir = temporaryFolder.newFolder();
File procDir = temporaryFolder.newFolder();
discoverer = new ProcCgroupDiscoverer(procDir.toPath());
TestUtils.setUpCgroups(procDir, cgroupDir);
cpusetDir = new File(
cgroupDir,
"cpuset/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
);
Assert.assertTrue((cpusetDir.isDirectory() && cpusetDir.exists()) || cpusetDir.mkdirs());
TestUtils.copyOrReplaceResource("/cpuset.cpus", new File(cpusetDir, "cpuset.cpus"));
TestUtils.copyOrReplaceResource("/cpuset.mems", new File(cpusetDir, "cpuset.mems"));
TestUtils.copyOrReplaceResource("/cpuset.effective_mems", new File(cpusetDir, "cpuset.effective_mems"));
}
@Test
public void testWontCrash()
{
final CpuSet cpuSet = new CpuSet(cgroup -> {
throw new RuntimeException("Should still continue");
});
final CpuSet.CpuSetMetric metric = cpuSet.snapshot();
Assert.assertEquals(0, metric.getCpuSetCpus().length);
Assert.assertEquals(0, metric.getEffectiveCpuSetCpus().length);
Assert.assertEquals(0, metric.getCpuSetMems().length);
Assert.assertEquals(0, metric.getEffectiveCpuSetMems().length);
}
@Test
public void testSimpleLoad() throws IOException
{
TestUtils.copyOrReplaceResource("/cpuset.effective_cpus.simple", new File(cpusetDir, "cpuset.effective_cpus"));
final CpuSet cpuSet = new CpuSet(discoverer);
final CpuSet.CpuSetMetric snapshot = cpuSet.snapshot();
Assert.assertArrayEquals(new int[]{0, 1, 2, 3, 4, 5, 6, 7}, snapshot.getCpuSetCpus());
Assert.assertArrayEquals(new int[]{0, 1, 2, 3, 4, 5, 6, 7}, snapshot.getEffectiveCpuSetCpus());
Assert.assertArrayEquals(new int[]{0, 1, 2, 3}, snapshot.getCpuSetMems());
Assert.assertArrayEquals(new int[]{0}, snapshot.getEffectiveCpuSetMems());
}
@Test
public void testComplexLoad() throws IOException
{
TestUtils.copyOrReplaceResource(
"/cpuset.effective_cpus.complex",
new File(cpusetDir, "cpuset.effective_cpus")
);
final CpuSet cpuSet = new CpuSet(discoverer);
final CpuSet.CpuSetMetric snapshot = cpuSet.snapshot();
Assert.assertArrayEquals(new int[]{0, 1, 2, 3, 4, 5, 6, 7}, snapshot.getCpuSetCpus());
Assert.assertArrayEquals(new int[]{0, 1, 2, 7, 12, 13, 14}, snapshot.getEffectiveCpuSetCpus());
Assert.assertArrayEquals(new int[]{0, 1, 2, 3}, snapshot.getCpuSetMems());
Assert.assertArrayEquals(new int[]{0}, snapshot.getEffectiveCpuSetMems());
}
}

View File

@ -0,0 +1,76 @@
/*
* 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.druid.java.util.metrics.cgroups;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
public class CpuTest
{
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private CgroupDiscoverer discoverer;
@Before
public void setUp() throws IOException
{
File cgroupDir = temporaryFolder.newFolder();
File procDir = temporaryFolder.newFolder();
discoverer = new ProcCgroupDiscoverer(procDir.toPath());
TestUtils.setUpCgroups(procDir, cgroupDir);
final File cpuDir = new File(
cgroupDir,
"cpu,cpuacct/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
);
Assert.assertTrue((cpuDir.isDirectory() && cpuDir.exists()) || cpuDir.mkdirs());
TestUtils.copyOrReplaceResource("/cpu.shares", new File(cpuDir, "cpu.shares"));
TestUtils.copyOrReplaceResource("/cpu.cfs_quota_us", new File(cpuDir, "cpu.cfs_quota_us"));
TestUtils.copyOrReplaceResource("/cpu.cfs_period_us", new File(cpuDir, "cpu.cfs_period_us"));
}
@Test
public void testWontCrash()
{
final Cpu cpu = new Cpu(cgroup -> {
throw new RuntimeException("Should still continue");
});
final Cpu.CpuAllocationMetric metric = cpu.snapshot();
Assert.assertEquals(-1L, metric.getShares());
Assert.assertEquals(0, metric.getQuotaUs());
Assert.assertEquals(0, metric.getPeriodUs());
}
@Test
public void testSimpleLoad()
{
final Cpu cpu = new Cpu(discoverer);
final Cpu.CpuAllocationMetric snapshot = cpu.snapshot();
Assert.assertEquals(1024, snapshot.getShares());
Assert.assertEquals(300000, snapshot.getQuotaUs());
Assert.assertEquals(100000, snapshot.getPeriodUs());
}
}

View File

@ -47,6 +47,11 @@ public class TestUtils
cgroupDir, cgroupDir,
"cpu,cpuacct/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee" "cpu,cpuacct/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
).mkdirs()); ).mkdirs());
Assert.assertTrue(new File(
cgroupDir,
"cpuset/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
).mkdirs());
copyResource("/proc.pid.cgroup", new File(procDir, "cgroup")); copyResource("/proc.pid.cgroup", new File(procDir, "cgroup"));
} }
@ -56,4 +61,11 @@ public class TestUtils
Assert.assertTrue(out.exists()); Assert.assertTrue(out.exists());
Assert.assertNotEquals(0, out.length()); Assert.assertNotEquals(0, out.length());
} }
public static void copyOrReplaceResource(String resource, File out) throws IOException
{
Files.copy(TestUtils.class.getResourceAsStream(resource), out.toPath());
Assert.assertTrue(out.exists());
Assert.assertNotEquals(0, out.length());
}
} }

View File

@ -0,0 +1 @@
ad1f0a5c-55ea-4a49-9db8-bbb0f22e2ba6

View File

@ -0,0 +1 @@
100000

View File

@ -0,0 +1 @@
300000

View File

@ -0,0 +1 @@
1024

View File

@ -0,0 +1,223 @@
processor : 0
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3293.170
cache size : 512 KB
physical id : 0
siblings : 8
core id : 0
cpu cores : 4
apicid : 0
initial apicid : 0
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 1
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3295.811
cache size : 512 KB
physical id : 0
siblings : 8
core id : 1
cpu cores : 4
apicid : 2
initial apicid : 2
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 2
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3294.112
cache size : 512 KB
physical id : 0
siblings : 8
core id : 2
cpu cores : 4
apicid : 4
initial apicid : 4
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 3
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3298.781
cache size : 512 KB
physical id : 0
siblings : 8
core id : 3
cpu cores : 4
apicid : 6
initial apicid : 6
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 4
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3296.715
cache size : 512 KB
physical id : 0
siblings : 8
core id : 0
cpu cores : 4
apicid : 1
initial apicid : 1
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 5
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3287.827
cache size : 512 KB
physical id : 0
siblings : 8
core id : 1
cpu cores : 4
apicid : 3
initial apicid : 3
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 6
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3297.435
cache size : 512 KB
physical id : 0
siblings : 8
core id : 2
cpu cores : 4
apicid : 5
initial apicid : 5
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:
processor : 7
vendor_id : AuthenticAMD
cpu family : 23
model : 49
model name : AMD EPYC 7R32
stepping : 0
microcode : 0x8301034
cpu MHz : 3296.936
cache size : 512 KB
physical id : 0
siblings : 8
core id : 3
cpu cores : 4
apicid : 7
initial apicid : 7
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt pdpe1gb rdtscp lm constant_tsc rep_good nopl nonstop_tsc cpuid extd_apicid aperfmperf tsc_known_freq pni pclmulqdq ssse3 fma cx16 sse4_1 sse4_2 movbe popcnt aes xsave avx f16c rdrand hypervisor lahf_lm cmp_legacy cr8_legacy abm sse4a misalignsse 3dnowprefetch topoext ssbd ibrs ibpb stibp vmmcall fsgsbase bmi1 avx2 smep bmi2 rdseed adx smap clflushopt clwb sha_ni xsaveopt xsavec xgetbv1 clzero xsaveerptr wbnoinvd arat npt nrip_save rdpid
bugs : sysret_ss_attrs spectre_v1 spectre_v2 spec_store_bypass
bogomips : 5600.00
TLB size : 3072 4K pages
clflush size : 64
cache_alignment : 64
address sizes : 48 bits physical, 48 bits virtual
power management:

View File

@ -0,0 +1 @@
0-7

View File

@ -0,0 +1 @@
0-2,7,12-14

View File

@ -0,0 +1 @@
0-7

View File

@ -0,0 +1 @@
0

View File

@ -0,0 +1 @@
0-3

View File

@ -5,7 +5,7 @@
7:devices:/system.slice/some.service 7:devices:/system.slice/some.service
6:freezer:/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee 6:freezer:/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee
5:hugetlb:/ 5:hugetlb:/
4:cpuset:/ 4:cpuset:/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee
3:memory:/system.slice/some.service 3:memory:/system.slice/some.service
2:net_cls,net_prio:/ 2:net_cls,net_prio:/
1:name=systemd:/mesos_executors.slice 1:name=systemd:/mesos_executors.slice