diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java b/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java new file mode 100644 index 00000000000..826465bd5ee --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java @@ -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 dimensions; + + public CgroupCpuMonitor(CgroupDiscoverer cgroupDiscoverer, final Map dimensions, String feed) + { + super(feed); + this.cgroupDiscoverer = cgroupDiscoverer; + this.dimensions = dimensions; + } + + public CgroupCpuMonitor(final Map dimensions, String feed) + { + this(null, dimensions, feed); + } + + public CgroupCpuMonitor(final Map 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; + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitor.java b/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitor.java new file mode 100644 index 00000000000..43a403ad5af --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitor.java @@ -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 dimensions; + + public CgroupCpuSetMonitor(CgroupDiscoverer cgroupDiscoverer, final Map dimensions, String feed) + { + super(feed); + this.cgroupDiscoverer = cgroupDiscoverer; + this.dimensions = dimensions; + } + + public CgroupCpuSetMonitor(final Map dimensions, String feed) + { + this(null, dimensions, feed); + } + + public CgroupCpuSetMonitor(final Map 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; + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/ProcFsReader.java b/core/src/main/java/org/apache/druid/java/util/metrics/ProcFsReader.java new file mode 100644 index 00000000000..1a1ca08c7c6 --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/metrics/ProcFsReader.java @@ -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 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 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; + } + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java b/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java new file mode 100644 index 00000000000..a742db2c3cb --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java @@ -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 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; + } + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/CpuSet.java b/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/CpuSet.java new file mode 100644 index 00000000000..31b343444dd --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/metrics/cgroups/CpuSet.java @@ -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 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; + } + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java b/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java new file mode 100644 index 00000000000..4a05f5fce6b --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java @@ -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 actualEvents = emitter.getEvents(); + Assert.assertEquals(2, actualEvents.size()); + final Map sharesEvent = actualEvents.get(0).toMap(); + final Map 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")); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitorTest.java b/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitorTest.java new file mode 100644 index 00000000000..c661355721e --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuSetMonitorTest.java @@ -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 actualEvents = emitter.getEvents(); + Assert.assertEquals(4, actualEvents.size()); + final Map cpusEvent = actualEvents.get(0).toMap(); + final Map effectiveCpusEvent = actualEvents.get(1).toMap(); + final Map memsEvent = actualEvents.get(2).toMap(); + final Map 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")); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/ProcFsReaderTest.java b/core/src/test/java/org/apache/druid/java/util/metrics/ProcFsReaderTest.java new file mode 100644 index 00000000000..0cd68d1bd7e --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/ProcFsReaderTest.java @@ -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()); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuSetTest.java b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuSetTest.java new file mode 100644 index 00000000000..516af9eb435 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuSetTest.java @@ -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()); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java new file mode 100644 index 00000000000..c2a023f3d39 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java @@ -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()); + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java index 1ba0bb136d1..c24b60d4fa5 100644 --- a/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java +++ b/core/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java @@ -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()); + } } diff --git a/core/src/test/resources/boot_id b/core/src/test/resources/boot_id new file mode 100644 index 00000000000..b0d4684d405 --- /dev/null +++ b/core/src/test/resources/boot_id @@ -0,0 +1 @@ +ad1f0a5c-55ea-4a49-9db8-bbb0f22e2ba6 \ No newline at end of file diff --git a/core/src/test/resources/cpu.cfs_period_us b/core/src/test/resources/cpu.cfs_period_us new file mode 100644 index 00000000000..483fb82b6dd --- /dev/null +++ b/core/src/test/resources/cpu.cfs_period_us @@ -0,0 +1 @@ +100000 \ No newline at end of file diff --git a/core/src/test/resources/cpu.cfs_quota_us b/core/src/test/resources/cpu.cfs_quota_us new file mode 100644 index 00000000000..247d42da613 --- /dev/null +++ b/core/src/test/resources/cpu.cfs_quota_us @@ -0,0 +1 @@ +300000 \ No newline at end of file diff --git a/core/src/test/resources/cpu.shares b/core/src/test/resources/cpu.shares new file mode 100644 index 00000000000..23cbfa77041 --- /dev/null +++ b/core/src/test/resources/cpu.shares @@ -0,0 +1 @@ +1024 \ No newline at end of file diff --git a/core/src/test/resources/cpuinfo b/core/src/test/resources/cpuinfo new file mode 100644 index 00000000000..c8df13121c3 --- /dev/null +++ b/core/src/test/resources/cpuinfo @@ -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: \ No newline at end of file diff --git a/core/src/test/resources/cpuset.cpus b/core/src/test/resources/cpuset.cpus new file mode 100644 index 00000000000..2ee6e60edd0 --- /dev/null +++ b/core/src/test/resources/cpuset.cpus @@ -0,0 +1 @@ +0-7 \ No newline at end of file diff --git a/core/src/test/resources/cpuset.effective_cpus.complex b/core/src/test/resources/cpuset.effective_cpus.complex new file mode 100644 index 00000000000..bfc70ba19f7 --- /dev/null +++ b/core/src/test/resources/cpuset.effective_cpus.complex @@ -0,0 +1 @@ +0-2,7,12-14 \ No newline at end of file diff --git a/core/src/test/resources/cpuset.effective_cpus.simple b/core/src/test/resources/cpuset.effective_cpus.simple new file mode 100644 index 00000000000..2ee6e60edd0 --- /dev/null +++ b/core/src/test/resources/cpuset.effective_cpus.simple @@ -0,0 +1 @@ +0-7 \ No newline at end of file diff --git a/core/src/test/resources/cpuset.effective_mems b/core/src/test/resources/cpuset.effective_mems new file mode 100644 index 00000000000..c227083464f --- /dev/null +++ b/core/src/test/resources/cpuset.effective_mems @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/core/src/test/resources/cpuset.mems b/core/src/test/resources/cpuset.mems new file mode 100644 index 00000000000..8b0ad1be2bd --- /dev/null +++ b/core/src/test/resources/cpuset.mems @@ -0,0 +1 @@ +0-3 \ No newline at end of file diff --git a/core/src/test/resources/proc.pid.cgroup b/core/src/test/resources/proc.pid.cgroup index e9ad851a940..179fb944f79 100644 --- a/core/src/test/resources/proc.pid.cgroup +++ b/core/src/test/resources/proc.pid.cgroup @@ -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