mirror of https://github.com/apache/druid.git
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:
parent
8fd17fe0af
commit
15789137a3
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -47,6 +47,11 @@ public class TestUtils
|
|||
cgroupDir,
|
||||
"cpu,cpuacct/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee"
|
||||
).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"));
|
||||
}
|
||||
|
||||
|
@ -56,4 +61,11 @@ public class TestUtils
|
|||
Assert.assertTrue(out.exists());
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
ad1f0a5c-55ea-4a49-9db8-bbb0f22e2ba6
|
|
@ -0,0 +1 @@
|
|||
100000
|
|
@ -0,0 +1 @@
|
|||
300000
|
|
@ -0,0 +1 @@
|
|||
1024
|
|
@ -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:
|
|
@ -0,0 +1 @@
|
|||
0-7
|
|
@ -0,0 +1 @@
|
|||
0-2,7,12-14
|
|
@ -0,0 +1 @@
|
|||
0-7
|
|
@ -0,0 +1 @@
|
|||
0
|
|
@ -0,0 +1 @@
|
|||
0-3
|
|
@ -5,7 +5,7 @@
|
|||
7:devices:/system.slice/some.service
|
||||
6:freezer:/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee
|
||||
5:hugetlb:/
|
||||
4:cpuset:/
|
||||
4:cpuset:/system.slice/some.service/f12ba7e0-fa16-462e-bb9d-652ccc27f0ee
|
||||
3:memory:/system.slice/some.service
|
||||
2:net_cls,net_prio:/
|
||||
1:name=systemd:/mesos_executors.slice
|
||||
|
|
Loading…
Reference in New Issue