8302883: JFR: Improve periodic events

Reviewed-by: mgronlun
This commit is contained in:
Erik Gahlin 2023-02-23 17:20:53 +00:00
parent a2471b37e3
commit 4b6acad0bd
20 changed files with 1125 additions and 351 deletions

View File

@ -44,8 +44,8 @@ import jdk.jfr.internal.Options;
import jdk.jfr.internal.PlatformRecorder;
import jdk.jfr.internal.PlatformRecording;
import jdk.jfr.internal.Repository;
import jdk.jfr.internal.RequestEngine;
import jdk.jfr.internal.Utils;
import jdk.jfr.internal.periodic.PeriodicEvents;
/**
* Class for accessing, controlling, and managing Flight Recorder.
@ -225,7 +225,7 @@ public final class FlightRecorder {
Utils.checkRegisterPermission();
@SuppressWarnings("removal")
AccessControlContext acc = AccessController.getContext();
RequestEngine.addHook(acc, EventType.getEventType(eventClass).getPlatformEventType(), hook);
PeriodicEvents.addUserEvent(acc, eventClass, hook);
}
/**
@ -242,7 +242,7 @@ public final class FlightRecorder {
if (JVMSupport.isNotAvailable()) {
return false;
}
return RequestEngine.removeHook(hook);
return PeriodicEvents.removeEvent(hook);
}
/**

View File

@ -48,7 +48,7 @@ public final class JVM {
* The monitor type is used to exclude jdk.JavaMonitorWait events from being generated
* when Object.wait() is called on this monitor.
*/
static final Object CHUNK_ROTATION_MONITOR = new ChunkRotationMonitor();
public static final Object CHUNK_ROTATION_MONITOR = new ChunkRotationMonitor();
private volatile boolean nativeOK;

View File

@ -46,9 +46,9 @@ import jdk.jfr.Period;
import jdk.jfr.StackTrace;
import jdk.jfr.Threshold;
import jdk.jfr.ValueDescriptor;
import jdk.jfr.internal.RequestEngine.RequestHook;
import jdk.jfr.internal.consumer.RepositoryFiles;
import jdk.jfr.internal.event.EventConfiguration;
import jdk.jfr.internal.periodic.PeriodicEvents;
public final class MetadataRepository {
@ -70,7 +70,6 @@ public final class MetadataRepository {
}
private void initializeJVMEventTypes() {
List<RequestHook> requestHooks = new ArrayList<>();
for (Type type : new ArrayList<>(typeLibrary.getTypes())) {
if (type instanceof PlatformEventType pEventType) {
EventType eventType = PrivateAccess.getInstance().newEventType(pEventType);
@ -84,14 +83,13 @@ public final class MetadataRepository {
if (pEventType.hasPeriod()) {
pEventType.setEventHook(true);
if (!pEventType.isMethodSampling()) {
requestHooks.add(new RequestHook(pEventType));
PeriodicEvents.addJVMEvent(pEventType);
}
}
nativeControls.add(new EventControl(pEventType));
nativeEventTypes.add(eventType);
}
}
RequestEngine.addHooks(requestHooks);
}
public static MetadataRepository getInstance() {

View File

@ -30,7 +30,7 @@ import java.util.List;
import java.util.Objects;
import jdk.jfr.SettingDescriptor;
import jdk.jfr.internal.periodic.PeriodicEvents;
/**
* Implementation of event type.
*
@ -65,7 +65,6 @@ public final class PlatformEventType extends Type {
private boolean registered = true;
private boolean committable = enabled && registered;
// package private
PlatformEventType(String name, long id, boolean isJDK, boolean dynamicSettings) {
super(name, Type.SUPER_TYPE_EVENT, id);
@ -201,6 +200,7 @@ public final class PlatformEventType extends Type {
}
public void setEnabled(boolean enabled) {
boolean changed = enabled != this.enabled;
this.enabled = enabled;
updateCommittable();
if (isJVM) {
@ -211,6 +211,9 @@ public final class PlatformEventType extends Type {
JVM.getJVM().setEnabled(getId(), enabled);
}
}
if (changed) {
PeriodicEvents.setChanged();
}
}
public void setPeriod(long periodMillis, boolean beginChunk, boolean endChunk) {
@ -220,7 +223,11 @@ public final class PlatformEventType extends Type {
}
this.beginChunk = beginChunk;
this.endChunk = endChunk;
boolean changed = period != periodMillis;
this.period = periodMillis;
if (changed) {
PeriodicEvents.setChanged();
}
}
public void setStackTraceEnabled(boolean stackTraceEnabled) {
@ -263,6 +270,7 @@ public final class PlatformEventType extends Type {
public void setEventHook(boolean hasHook) {
this.hasHook = hasHook;
PeriodicEvents.setChanged();
}
public boolean isBeginChunk() {

View File

@ -57,6 +57,7 @@ import jdk.jfr.internal.SecuritySupport.SafePath;
import jdk.jfr.internal.SecuritySupport.SecureRecorderListener;
import jdk.jfr.internal.consumer.EventLog;
import jdk.jfr.internal.instrument.JDKEvents;
import jdk.jfr.internal.periodic.PeriodicEvents;
public final class PlatformRecorder {
@ -258,7 +259,7 @@ public final class PlatformRecorder {
if (EventLog.shouldLog()) {
EventLog.start();
}
RequestEngine.doChunkEnd();
PeriodicEvents.doChunkEnd();
String p = newChunk.getFile().toString();
startTime = MetadataRepository.getInstance().setOutput(p);
newChunk.setStartTime(startTime);
@ -275,9 +276,9 @@ public final class PlatformRecorder {
currentChunk = newChunk;
}
if (toDisk) {
RequestEngine.setFlushInterval(streamInterval);
PeriodicEvents.setFlushInterval(streamInterval);
}
RequestEngine.doChunkBegin();
PeriodicEvents.doChunkBegin();
Duration duration = recording.getDuration();
if (duration != null) {
recording.setStopTime(startTime.plus(duration));
@ -313,7 +314,7 @@ public final class PlatformRecorder {
recording.setFinalStartnanos(Utils.getChunkStartNanos());
if (endPhysical) {
RequestEngine.doChunkEnd();
PeriodicEvents.doChunkEnd();
if (recording.isToDisk()) {
if (inShutdown) {
jvm.markChunkFinal();
@ -330,7 +331,7 @@ public final class PlatformRecorder {
disableEvents();
} else {
RepositoryChunk newChunk = null;
RequestEngine.doChunkEnd();
PeriodicEvents.doChunkEnd();
updateSettingsButIgnoreRecording(recording, false);
String path = null;
@ -348,13 +349,13 @@ public final class PlatformRecorder {
finishChunk(currentChunk, stopTime, null);
}
currentChunk = newChunk;
RequestEngine.doChunkBegin();
PeriodicEvents.doChunkBegin();
}
if (toDisk) {
RequestEngine.setFlushInterval(streamInterval);
PeriodicEvents.setFlushInterval(streamInterval);
} else {
RequestEngine.setFlushInterval(Long.MAX_VALUE);
PeriodicEvents.setFlushInterval(Long.MAX_VALUE);
}
recording.setState(RecordingState.STOPPED);
if (!isToDisk()) {
@ -394,7 +395,7 @@ public final class PlatformRecorder {
synchronized void rotateDisk() {
RepositoryChunk newChunk = repository.newChunk();
RequestEngine.doChunkEnd();
PeriodicEvents.doChunkEnd();
String path = newChunk.getFile().toString();
Instant timestamp = MetadataRepository.getInstance().setOutput(path);
newChunk.setStartTime(timestamp);
@ -403,7 +404,7 @@ public final class PlatformRecorder {
finishChunk(currentChunk, timestamp, null);
}
currentChunk = newChunk;
RequestEngine.doChunkBegin();
PeriodicEvents.doChunkBegin();
}
private List<PlatformRecording> getRunningRecordings() {
@ -499,7 +500,7 @@ public final class PlatformRecorder {
EventLog.update();
}
}
long minDelta = RequestEngine.doPeriodic();
long minDelta = PeriodicEvents.doPeriodic();
long wait = Math.min(minDelta, Options.getWaitInterval());
takeNap(wait);
}

View File

@ -1,311 +0,0 @@
/*
* Copyright (c) 2016, 2022, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal;
import java.security.AccessControlContext;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import jdk.jfr.Event;
import jdk.jfr.EventType;
public final class RequestEngine {
enum PeriodicType {
BEGIN_CHUNK, INTERVAL, END_CHUNK
}
private static final JVM jvm = JVM.getJVM();
private static final ReentrantLock lock = new ReentrantLock();
static final class RequestHook {
private final Runnable hook;
private final PlatformEventType type;
@SuppressWarnings("removal")
private final AccessControlContext accessControllerContext;
private long delta;
// Java events
private RequestHook(@SuppressWarnings("removal") AccessControlContext acc, PlatformEventType eventType, Runnable hook) {
this.hook = hook;
this.type = eventType;
this.accessControllerContext = acc;
}
// native events
RequestHook(PlatformEventType eventType) {
this(null, eventType, null);
}
private void execute(long timestamp, PeriodicType periodicType) {
try {
if (accessControllerContext == null) { // native
if (type.isJDK()) {
hook.run();
} else {
emitJVMEvent(type, timestamp, periodicType);
}
if (Logger.shouldLog(LogTag.JFR_SYSTEM, LogLevel.DEBUG)) {
Logger.log(LogTag.JFR_SYSTEM, LogLevel.DEBUG, "Executed periodic hook for " + type.getLogName());
}
} else {
executeSecure();
}
} catch (Throwable e) {
// Prevent malicious user to propagate exception callback in the wrong context
Logger.log(LogTag.JFR_SYSTEM, LogLevel.WARN, "Exception occurred during execution of period hook for " + type.getLogName());
}
}
private void emitJVMEvent(PlatformEventType type, long timestamp, PeriodicType periodicType) {
try {
// There should only be one thread in native at a time.
// ReentrantLock is used to avoid JavaMonitorBlocked event
// from synchronized block.
lock.lock();
jvm.emitEvent(type.getId(), timestamp, periodicType.ordinal());
} finally {
lock.unlock();
}
}
@SuppressWarnings("removal")
private void executeSecure() {
AccessController.doPrivileged(new PrivilegedAction<Void>() {
@Override
public Void run() {
try {
hook.run();
if (Logger.shouldLog(LogTag.JFR_EVENT, LogLevel.DEBUG)) {
Logger.log(LogTag.JFR_EVENT, LogLevel.DEBUG, "Executed periodic hook for " + type.getLogName());
}
} catch (Throwable t) {
// Prevent malicious user to propagate exception callback in the wrong context
Logger.log(LogTag.JFR_EVENT, LogLevel.WARN, "Exception occurred during execution of period hook for " + type.getLogName());
}
return null;
}
}, accessControllerContext);
}
}
private static final List<RequestHook> entries = new CopyOnWriteArrayList<>();
private static long lastTimeMillis;
private static long flushInterval = Long.MAX_VALUE;
private static long streamDelta;
public static void addHook(@SuppressWarnings("removal") AccessControlContext acc, PlatformEventType type, Runnable hook) {
Objects.requireNonNull(acc);
addHookInternal(acc, type, hook);
}
private static void addHookInternal(@SuppressWarnings("removal") AccessControlContext acc, PlatformEventType type, Runnable hook) {
RequestHook he = new RequestHook(acc, type, hook);
for (RequestHook e : entries) {
if (e.hook == hook) {
throw new IllegalArgumentException("Hook has already been added");
}
}
he.type.setEventHook(true);
// Insertion takes O(2*n), could be O(1) with HashMap, but
// thinking is that CopyOnWriteArrayList is faster
// to iterate over, which will happen more over time.
entries.add(he);
logHook("Added", type);
}
public static void addTrustedJDKHook(Class<? extends Event> eventClass, Runnable runnable) {
if (eventClass.getClassLoader() != null) {
throw new SecurityException("Hook can only be registered for event classes that are loaded by the bootstrap class loader");
}
if (runnable.getClass().getClassLoader() != null) {
throw new SecurityException("Runnable hook class must be loaded by the bootstrap class loader");
}
EventType eType = MetadataRepository.getInstance().getEventType(eventClass);
PlatformEventType pType = PrivateAccess.getInstance().getPlatformEventType(eType);
addHookInternal(null, pType, runnable);
}
private static void logHook(String action, PlatformEventType type) {
if (type.isSystem()) {
Logger.log(LogTag.JFR_SYSTEM, LogLevel.INFO, action + " periodic hook for " + type.getLogName());
} else {
Logger.log(LogTag.JFR, LogLevel.INFO, action + " periodic hook for " + type.getLogName());
}
}
// Takes O(2*n), see addHook.
public static boolean removeHook(Runnable hook) {
for (RequestHook rh : entries) {
if (rh.hook == hook) {
entries.remove(rh);
rh.type.setEventHook(false);
logHook("Removed", rh.type);
return true;
}
}
return false;
}
// Only to be used for JVM events. No access control contest
// or check if hook already exists
static void addHooks(List<RequestHook> newEntries) {
for (RequestHook rh : newEntries) {
rh.type.setEventHook(true);
logHook("Added", rh.type);
}
entries.addAll(newEntries);
}
static void doChunkEnd() {
doChunk(x -> x.isEndChunk(), PeriodicType.END_CHUNK);
}
static void doChunkBegin() {
doChunk(x -> x.isBeginChunk(), PeriodicType.BEGIN_CHUNK);
}
private static void doChunk(Predicate<PlatformEventType> predicate, PeriodicType type) {
long timestamp = JVM.counterTime();
for (RequestHook requestHook : entries) {
PlatformEventType s = requestHook.type;
if (s.isEnabled() && predicate.test(s)) {
requestHook.execute(timestamp, type);
}
}
}
static long doPeriodic() {
return run_requests(entries, JVM.counterTime());
}
// code copied from native impl.
private static long run_requests(Collection<RequestHook> entries, long eventTimestamp) {
long last = lastTimeMillis;
// The interval for periodic events is typically at least 1 s, so
// System.currentTimeMillis() is sufficient. JVM.counterTime() lacks
// unit and has in the past been more unreliable.
long now = System.currentTimeMillis();
long min = 0;
long delta = 0;
if (last == 0) {
last = now;
}
// time from then to now
delta = now - last;
if (delta < 0) {
// to handle time adjustments
// for example Daylight Savings
lastTimeMillis = now;
return 0;
}
Iterator<RequestHook> hookIterator = entries.iterator();
while(hookIterator.hasNext()) {
RequestHook he = hookIterator.next();
long left = 0;
PlatformEventType es = he.type;
// Not enabled, skip.
if (!es.isEnabled() || es.isChunkTime()) {
continue;
}
long r_period = es.getPeriod();
long r_delta = he.delta;
// add time elapsed.
r_delta += delta;
// above threshold?
if (r_delta >= r_period) {
// Bug 9000556 - don't try to compensate
// for wait > period
r_delta = 0;
he.execute(eventTimestamp, PeriodicType.INTERVAL);
}
// calculate time left
left = (r_period - r_delta);
/*
* nothing outside checks that a period is >= 0, so left can end up
* negative here. ex. (r_period =(-1)) - (r_delta = 0) if it is,
* handle it.
*/
if (left < 0) {
left = 0;
}
// assign delta back
he.delta = r_delta;
if (min == 0 || left < min) {
min = left;
}
}
// Flush should happen after all periodic events has been emitted
// Repeat of the above algorithm, but using the stream interval.
if (flushInterval != Long.MAX_VALUE) {
long r_period = flushInterval;
long r_delta = streamDelta;
r_delta += delta;
if (r_delta >= r_period) {
r_delta = 0;
MetadataRepository.getInstance().flush();
Utils.notifyFlush();
}
long left = (r_period - r_delta);
if (left < 0) {
left = 0;
}
streamDelta = r_delta;
if (min == 0 || left < min) {
min = left;
}
}
lastTimeMillis = now;
return min;
}
static void setFlushInterval(long millis) {
// Don't accept shorter interval than 1 s.
long interval = millis < 1000 ? 1000 : millis;
boolean needNotify = interval < flushInterval;
flushInterval = interval;
if (needNotify) {
synchronized (JVM.CHUNK_ROTATION_MONITOR) {
JVM.CHUNK_ROTATION_MONITOR.notifyAll();
}
}
}
}

View File

@ -64,9 +64,8 @@ import jdk.jfr.internal.JVM;
import jdk.jfr.internal.LogLevel;
import jdk.jfr.internal.LogTag;
import jdk.jfr.internal.Logger;
import jdk.jfr.internal.RequestEngine;
import jdk.jfr.internal.SecuritySupport;
import jdk.jfr.internal.periodic.PeriodicEvents;
import jdk.internal.platform.Container;
import jdk.internal.platform.Metrics;
@ -148,10 +147,9 @@ public final class JDKEvents {
for (Class<?> eventClass : eventClasses) {
SecuritySupport.registerEvent((Class<? extends Event>) eventClass);
}
RequestEngine.addTrustedJDKHook(ExceptionStatisticsEvent.class, emitExceptionStatistics);
RequestEngine.addTrustedJDKHook(DirectBufferStatisticsEvent.class, emitDirectBufferStatistics);
RequestEngine.addTrustedJDKHook(InitialSecurityPropertyEvent.class, emitInitialSecurityProperties);
PeriodicEvents.addJDKEvent(ExceptionStatisticsEvent.class, emitExceptionStatistics);
PeriodicEvents.addJDKEvent(DirectBufferStatisticsEvent.class, emitDirectBufferStatistics);
PeriodicEvents.addJDKEvent(InitialSecurityPropertyEvent.class, emitInitialSecurityProperties);
initializeContainerEvents();
initializationTriggered = true;
@ -197,11 +195,11 @@ public final class JDKEvents {
SecuritySupport.registerEvent(ContainerMemoryUsageEvent.class);
SecuritySupport.registerEvent(ContainerIOUsageEvent.class);
RequestEngine.addTrustedJDKHook(ContainerConfigurationEvent.class, emitContainerConfiguration);
RequestEngine.addTrustedJDKHook(ContainerCPUUsageEvent.class, emitContainerCPUUsage);
RequestEngine.addTrustedJDKHook(ContainerCPUThrottlingEvent.class, emitContainerCPUThrottling);
RequestEngine.addTrustedJDKHook(ContainerMemoryUsageEvent.class, emitContainerMemoryUsage);
RequestEngine.addTrustedJDKHook(ContainerIOUsageEvent.class, emitContainerIOUsage);
PeriodicEvents.addJDKEvent(ContainerConfigurationEvent.class, emitContainerConfiguration);
PeriodicEvents.addJDKEvent(ContainerCPUUsageEvent.class, emitContainerCPUUsage);
PeriodicEvents.addJDKEvent(ContainerCPUThrottlingEvent.class, emitContainerCPUThrottling);
PeriodicEvents.addJDKEvent(ContainerMemoryUsageEvent.class, emitContainerMemoryUsage);
PeriodicEvents.addJDKEvent(ContainerIOUsageEvent.class, emitContainerIOUsage);
}
private static void emitExceptionStatistics() {
@ -293,15 +291,15 @@ public final class JDKEvents {
}
public static void remove() {
RequestEngine.removeHook(emitExceptionStatistics);
RequestEngine.removeHook(emitDirectBufferStatistics);
RequestEngine.removeHook(emitInitialSecurityProperties);
PeriodicEvents.removeEvent(emitExceptionStatistics);
PeriodicEvents.removeEvent(emitDirectBufferStatistics);
PeriodicEvents.removeEvent(emitInitialSecurityProperties);
RequestEngine.removeHook(emitContainerConfiguration);
RequestEngine.removeHook(emitContainerCPUUsage);
RequestEngine.removeHook(emitContainerCPUThrottling);
RequestEngine.removeHook(emitContainerMemoryUsage);
RequestEngine.removeHook(emitContainerIOUsage);
PeriodicEvents.removeEvent(emitContainerConfiguration);
PeriodicEvents.removeEvent(emitContainerCPUUsage);
PeriodicEvents.removeEvent(emitContainerCPUThrottling);
PeriodicEvents.removeEvent(emitContainerMemoryUsage);
PeriodicEvents.removeEvent(emitContainerIOUsage);
}
private static void emitDirectBufferStatistics() {

View File

@ -0,0 +1,76 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.util.ArrayList;
import java.util.List;
/**
* Class that holds periodic tasks that run at the same time.
* <p>
* For example, events with period 1s, 3s and 7s can run when the 1s event run,
* not every time, but some of the time. An event with period 1.5s would not
* belong to the same batch since it would need to run between the 1s interval.
* <p>
* This class should only be accessed from the periodic task thread.
*/
final class Batch {
private final List<PeriodicTask> tasks = new ArrayList<>();
private final long period;
private long delta;
public Batch(long period) {
this.period = period;
}
public long getDelta() {
return delta;
}
public void setDelta(long delta) {
this.delta = delta;
}
public long getPeriod() {
return period;
}
public List<PeriodicTask> getTasks() {
return tasks;
}
public void add(PeriodicTask task) {
task.setBatch(this);
tasks.add(task);
}
public void clear() {
tasks.clear();
}
public boolean isEmpty() {
return tasks.isEmpty();
}
}

View File

@ -0,0 +1,125 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import jdk.jfr.internal.LogLevel;
import jdk.jfr.internal.LogTag;
import jdk.jfr.internal.Logger;
/**
* Class that groups periodic tasks into batches.
* <p>
* This class should only be accessed from the periodic task thread.
*/
final class BatchManager {
private final List<Batch> batches = new ArrayList<>();
private long iteration = -1;
public List<Batch> getBatches() {
return batches;
}
public long getIteration() {
return iteration;
}
public void refresh(long iteration, List<PeriodicTask> tasks) {
groupTasksIntoBatches(tasks);
this.iteration = iteration;
logBatches();
}
private void groupTasksIntoBatches(List<PeriodicTask> tasks) {
// Batches are cleared instead of recreated to keep batch delta intact
for (Batch batch : batches) {
batch.clear();
}
for (PeriodicTask task : activeSortedTasks(tasks)) {
if (task.isSchedulable()) {
Batch batch = task.getBatch();
// If new task, or period has changed, find new batch
if (batch == null) {
batch = findBatch(task.getPeriod());
}
batch.add(task);
}
}
// Remove unused batches
batches.removeIf(Batch::isEmpty);
}
private List<PeriodicTask> activeSortedTasks(List<PeriodicTask> unsorted) {
// Update with latest periods
List<PeriodicTask> tasks = new ArrayList<>(unsorted.size());
for (PeriodicTask task : unsorted) {
task.updatePeriod();
if (task.getPeriod() != 0) {
tasks.add(task);
}
}
// Sort tasks by lowest period
tasks.sort(Comparator.comparingLong(PeriodicTask::getPeriod));
return tasks;
}
private Batch findBatch(long period) {
// All events with a period less than 1000 ms
// get their own unique batch. The rationale for
// this is to avoid a scenario where a user (mistakenly) specifies
// period=1ms for an event and then all events end
// up in that batch. It would work, but 99,9% of the time
// the iteration would be pointless.
for (Batch batch : batches) {
long batchPeriod = batch.getPeriod();
if ((period >= 1000 && batchPeriod >= 1000 && period % batchPeriod == 0) || (batchPeriod == period)) {
return batch;
}
}
Batch batch = new Batch(period);
batches.add(batch);
return batch;
}
private void logBatches() {
if (!Logger.shouldLog(LogTag.JFR, LogLevel.TRACE)) {
return;
}
String prefix = "Periodic task: settings iteration: " + iteration + ", batch period: ";
for (Batch batch : batches) {
String batchPrefix = prefix + batch.getPeriod();
for (PeriodicTask task : batch.getTasks()) {
logTrace(batchPrefix + ", period: " + task.getPeriod() + ", task: " + task.getName());
}
}
}
private void logTrace(String text) {
Logger.log(LogTag.JFR_SYSTEM, LogLevel.TRACE, text);
}
}

View File

@ -0,0 +1,53 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import jdk.jfr.internal.PlatformEventType;
/**
* Base class for periodic events.
*/
abstract class EventTask extends PeriodicTask {
private final PlatformEventType eventType;
public EventTask(PlatformEventType eventType, LookupKey lookupKey) {
super(lookupKey, eventType.getLogName());
this.eventType = eventType;
}
@Override
public final boolean isSchedulable() {
return eventType.isEnabled() && !eventType.isChunkTime();
}
@Override
protected final long fetchPeriod() {
return eventType.getPeriod();
}
public final PlatformEventType getEventType() {
return eventType;
}
}

View File

@ -0,0 +1,73 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import jdk.jfr.internal.JVM;
import jdk.jfr.internal.MetadataRepository;
import jdk.jfr.internal.Utils;
/**
* Periodic task that flushes event data to disk.
*<p>
* The task is run once every second and after all other periodic events.
* <p>
* A flush interval of {@code Long.MAX_VALUE} means the event is disabled.
*/
final class FlushTask extends PeriodicTask {
private volatile long flushInterval = Long.MAX_VALUE;
public FlushTask() {
super(new LookupKey(new Object()), "JFR: Flush Task");
}
@Override
public void execute(long timestamp, PeriodicType periodicType) {
MetadataRepository.getInstance().flush();
Utils.notifyFlush();
}
@Override
public boolean isSchedulable() {
return true;
}
@Override
protected long fetchPeriod() {
return flushInterval;
}
public void setInterval(long millis) {
// Don't accept shorter interval than 1 s
long interval = millis < 1000 ? 1000 : millis;
boolean needsNotify = interval < flushInterval;
flushInterval = interval;
PeriodicEvents.setChanged();
if (needsNotify) {
synchronized (JVM.CHUNK_ROTATION_MONITOR) {
JVM.CHUNK_ROTATION_MONITOR.notifyAll();
}
}
}
}

View File

@ -0,0 +1,54 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import jdk.jfr.Event;
/**
* Periodic task that runs trusted code that doesn't require an access control
* context.
* <p>
* This class can be removed once the Security Manager is no longer supported.
*/
final class JDKEventTask extends JavaEventTask {
public JDKEventTask(Class<? extends Event> eventClass, Runnable runnable) {
super(eventClass, runnable);
if (!getEventType().isJDK()) {
throw new InternalError("Must be a JDK event");
}
if (eventClass.getClassLoader() != null) {
throw new SecurityException("Periodic task can only be registered for event classes that are loaded by the bootstrap class loader");
}
if (runnable.getClass().getClassLoader() != null) {
throw new SecurityException("Runnable class must be loaded by the bootstrap class loader");
}
}
@Override
public void execute(long timestamp, PeriodicType periodicType) {
getRunnable().run();
}
}

View File

@ -0,0 +1,59 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import jdk.jfr.internal.JVM;
import jdk.jfr.internal.PlatformEventType;
/**
* Task for periodic events defined in the JVM.
* <p>
* This class guarantees that only one event can execute in native at a time.
*/
final class JVMEventTask extends EventTask {
// java.util.concurrent lock is used to avoid JavaMonitorBlocked event from
// synchronized block.
private static final Lock lock = new ReentrantLock();
public JVMEventTask(PlatformEventType eventType) {
super(eventType, new LookupKey(eventType));
if (!eventType.isJVM()) {
throw new InternalError("Must be a JVM event");
}
}
@Override
public void execute(long timestamp, PeriodicType periodicType) {
try {
lock.lock();
JVM.getJVM().emitEvent(getEventType().getId(), timestamp, periodicType.ordinal());
} finally {
lock.unlock();
}
}
}

View File

@ -0,0 +1,54 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import jdk.jfr.Event;
import jdk.jfr.EventType;
import jdk.jfr.internal.MetadataRepository;
import jdk.jfr.internal.PlatformEventType;
import jdk.jfr.internal.PrivateAccess;
/**
* Base class for periodic Java events.
*/
abstract class JavaEventTask extends EventTask {
private final Runnable runnable;
public JavaEventTask(Class<? extends Event> eventClass, Runnable runnable) {
super(toPlatformEventType(eventClass), new LookupKey(runnable));
this.runnable = runnable;
if (getEventType().isJVM()) {
throw new InternalError("Must not be a JVM event");
}
}
private static PlatformEventType toPlatformEventType(Class<? extends Event> eventClass) {
EventType eventType = MetadataRepository.getInstance().getEventType(eventClass);
return PrivateAccess.getInstance().getPlatformEventType(eventType);
}
protected final Runnable getRunnable() {
return runnable;
}
}

View File

@ -0,0 +1,53 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
/**
* Lookup key that can safely be used in a {@code Map}.
* <p>
* {@code Runnable} objects can't be used with {@code LinkedHashMap} as it
* invokes {@code hashCode} and {@code equals}, for example when resizing the
* {@code Map}, possibly in a non-secure context.
* <p>
* {@code IdentityHashMap} can't be used as it will not preserve order.
*/
final class LookupKey {
private final Object object;
public LookupKey(Object object) {
this.object = object;
}
public int hashCode() {
return System.identityHashCode(object);
}
public boolean equals(Object that) {
if (that instanceof LookupKey lookupKey) {
return lookupKey.object == object;
}
return false;
}
}

View File

@ -0,0 +1,208 @@
/*
* Copyright (c) 2016, 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.security.AccessControlContext;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import jdk.jfr.Event;
import jdk.jfr.internal.JVM;
import jdk.jfr.internal.PlatformEventType;
/**
* Class that runs and schedules tasks for periodic events.
* <p>
* Events can run at the beginning of a chunk rotation, at the end of a chunk
* rotation or at a periodic interval.
* <p>
* Events with the same period runs in the same order as users added them.
* Temporarily disabling events don't impact the order of execution.
* <p>
* A best effort is made to run events with different periods at the same time.
* For example, an event that runs every two seconds executes half of the time
* with the events that run every second.
*/
public final class PeriodicEvents {
private static final TaskRepository taskRepository = new TaskRepository();
private static final BatchManager batchManager = new BatchManager();
private static final FlushTask flushTask = new FlushTask();
private static final AtomicLong settingsIteration = new AtomicLong();
// State only to be read and modified by periodic task thread
private static long lastTimeMillis;
public static void addJDKEvent(Class<? extends Event> eventClass, Runnable runnable) {
taskRepository.add(new JDKEventTask(eventClass, runnable));
}
public static void addJVMEvent(PlatformEventType eventType) {
taskRepository.add(new JVMEventTask(eventType));
}
public static void addUserEvent(@SuppressWarnings("removal") AccessControlContext acc, Class<? extends Event> eventClass, Runnable runnable) {
taskRepository.add(new UserEventTask(acc, eventClass, runnable));
}
public static boolean removeEvent(Runnable runnable) {
return taskRepository.removeTask(runnable);
}
public static void doChunkBegin() {
long timestamp = JVM.counterTime();
for (EventTask task : taskRepository.getTasks()) {
var eventType = task.getEventType();
if (eventType.isEnabled() && eventType.isBeginChunk()) {
task.run(timestamp, PeriodicType.BEGIN_CHUNK);
}
}
}
public static void doChunkEnd() {
long timestamp = JVM.counterTime();
for (EventTask task : taskRepository.getTasks()) {
var eventType = task.getEventType();
if (eventType.isEnabled() && eventType.isEndChunk()) {
task.run(timestamp, PeriodicType.END_CHUNK);
}
}
}
// Only to be called from periodic task thread
public static long doPeriodic() {
try {
return runPeriodic(JVM.counterTime());
} catch (Throwable t) {
t.printStackTrace();
throw t;
}
}
// Code copied from prior native implementation
private static long runPeriodic(long eventTimestamp) {
long last = lastTimeMillis;
// The interval for periodic events is typically at least 1 s, so
// System.currentTimeMillis() is sufficient. JVM.counterTime() lacks
// unit and has in the past been more unreliable.
long now = System.currentTimeMillis();
long min = 0;
long delta = 0;
if (last == 0) {
last = now;
}
// time from then to now
delta = now - last;
if (delta < 0) {
// to handle time adjustments
// for example Daylight Savings
lastTimeMillis = now;
return 0;
}
long iteration = settingsIteration.get();
if (iteration > batchManager.getIteration()) {
List<PeriodicTask> tasks = new ArrayList<>();
tasks.addAll(taskRepository.getTasks());
tasks.add(flushTask);
batchManager.refresh(iteration, tasks);
}
boolean flush = false;
for (Batch batch : batchManager.getBatches()) {
long left = 0;
long r_period = batch.getPeriod();
long r_delta = batch.getDelta();
// add time elapsed.
r_delta += delta;
// above threshold?
if (r_delta >= r_period) {
// Bug 9000556 - don't try to compensate
// for wait > period
r_delta = 0;
for (PeriodicTask task : batch.getTasks()) {
task.tick();
if (task.shouldRun()) {
if (task instanceof FlushTask) {
flush = true;
} else {
task.run(eventTimestamp, PeriodicType.INTERVAL);
}
}
}
}
// calculate time left
left = (r_period - r_delta);
/*
* nothing outside checks that a period is >= 0, so left can end up negative
* here. ex. (r_period =(-1)) - (r_delta = 0) if it is, handle it.
*/
if (left < 0) {
left = 0;
}
// assign delta back
batch.setDelta(r_delta);
if (min == 0 || left < min) {
min = left;
}
}
if (flush) {
flushTask.run(eventTimestamp, PeriodicType.INTERVAL);
}
lastTimeMillis = now;
return min;
}
/**
* Marks that a change has happened to a periodic event.
* <p>
* This method should be invoked if a periodic event has:
* <ul>
* <li>been added</li>
* <li>been removed</li>
* <li>been enabled
* <li>been disabled</li>
* <li>changed period</li>
* </ul>
* <p>
* The periodic task thread will poll the changed state at least once every
* second to see if a change has occurred. if that's the case, it will refresh
* periodic tasks that need to be run.
*/
public static void setChanged() {
settingsIteration.incrementAndGet();
}
public static void setFlushInterval(long millis) {
flushTask.setInterval(millis);
}
}

View File

@ -0,0 +1,127 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import jdk.jfr.internal.LogLevel;
import jdk.jfr.internal.LogTag;
import jdk.jfr.internal.Logger;
/**
* Base class that holds time related information for a periodic task.
* <p>
* Class hierarchy for periodic tasks:
* <pre>
* PeriodicTask
* / \
* / \
* EventTask FlushTask
* / \
* / \
* JVMEventTask JavaEventTask
* / \
* / \
* UserEventTask JDKEventTask
* </pre>
* <p>
* State modifications should only be done from the periodic task thread.
*/
abstract class PeriodicTask {
private final LookupKey lookupKey;
private final String name;
// State only to be modified by the periodic task thread
private long counter;
private long period;
private Batch batch;
public PeriodicTask(LookupKey lookupKey, String name) {
this.lookupKey = lookupKey;
this.name = name;
}
public abstract void execute(long timestamp, PeriodicType periodicType);
public abstract boolean isSchedulable();
protected abstract long fetchPeriod();
public final LookupKey getLookupKey() {
return lookupKey;
}
public final String getName() {
return name;
}
// Only to be called from periodic task thread
public void setBatch(Batch batch) {
this.batch = batch;
}
// Only to be called from periodic task thread
public Batch getBatch() {
return batch;
}
// Only to be called from periodic task thread
public final void tick() {
long increment = batch.getPeriod();
if (period != 0) {
counter = (counter + increment) % period;
}
}
// Only to be called from periodic task thread
public final boolean shouldRun() {
return counter == 0 && period != 0;
}
// Only to be called from periodic task thread
public final void updatePeriod() {
long p = fetchPeriod();
// Reset counter if new period
if (p != period) {
counter = 0;
period = p;
batch = null;
}
}
// Only to be called from periodic task thread
public final long getPeriod() {
return period;
}
public final void run(long timestamp, PeriodicType periodicType) {
try {
execute(timestamp, periodicType);
} catch (Throwable e) {
// Prevent malicious user to propagate exception callback in the wrong context
Logger.log(LogTag.JFR_SYSTEM, LogLevel.WARN, "Exception occurred during execution of period task for " + name);
}
if (Logger.shouldLog(LogTag.JFR_SYSTEM, LogLevel.DEBUG)) {
Logger.log(LogTag.JFR_SYSTEM, LogLevel.DEBUG, "Executed periodic task for " + name);
}
}
}

View File

@ -0,0 +1,40 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
enum PeriodicType {
/**
* Event is running at the beginning of a chunk rotation.
*/
BEGIN_CHUNK,
/**
* Event is running at an interval, for example, once every second.
*/
INTERVAL,
/**
* Event is running at the end of a chunk rotation.
*/
END_CHUNK
}

View File

@ -0,0 +1,87 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import jdk.jfr.internal.LogLevel;
import jdk.jfr.internal.LogTag;
import jdk.jfr.internal.Logger;
import jdk.jfr.internal.PlatformEventType;
/**
* Class that holds periodic tasks.
* <p>
* This class is thread safe.
*/
final class TaskRepository {
// Keeps periodic tasks in the order they were added by the user
private final Map<LookupKey, EventTask> lookup = new LinkedHashMap<>();
// An immutable copy that can be used to iterate over tasks.
private List<EventTask> cache;
public synchronized List<EventTask> getTasks() {
if (cache == null) {
cache = List.copyOf(lookup.values());
}
return cache;
}
public synchronized boolean removeTask(Runnable r) {
EventTask pt = lookup.remove(new LookupKey(r));
if (pt != null) {
var eventType = pt.getEventType();
// Invokes PeriodicEvents.setChanged()
eventType.setEventHook(false);
logTask("Removed", eventType);
cache = null;
return true;
}
return false;
}
public synchronized void add(EventTask task) {
if (lookup.containsKey(task.getLookupKey())) {
throw new IllegalArgumentException("Hook has already been added");
}
lookup.put(task.getLookupKey(), task);
var eventType = task.getEventType();
// Invokes PeriodicEvents.setChanged()
eventType.setEventHook(true);
logTask("Added", eventType);
cache = null;
}
private void logTask(String action, PlatformEventType type) {
if (type.isSystem()) {
Logger.log(LogTag.JFR_SYSTEM, LogLevel.INFO, action + " periodic task for " + type.getLogName());
} else {
Logger.log(LogTag.JFR, LogLevel.INFO, action + " periodic task for " + type.getLogName());
}
}
}

View File

@ -0,0 +1,71 @@
/*
* Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
package jdk.jfr.internal.periodic;
import java.security.AccessControlContext;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.Objects;
import jdk.jfr.Event;
import jdk.jfr.internal.LogLevel;
import jdk.jfr.internal.LogTag;
import jdk.jfr.internal.Logger;
/**
* Class to be used with user-defined events that runs untrusted code.
* <p>
* This class can be removed once the Security Manager is no longer supported.
*/
final class UserEventTask extends JavaEventTask {
@SuppressWarnings("removal")
private final AccessControlContext controlContext;
public UserEventTask(@SuppressWarnings("removal") AccessControlContext controlContext, Class<? extends Event> eventClass, Runnable runnable) {
super(eventClass, runnable);
this.controlContext = Objects.requireNonNull(controlContext);
}
@SuppressWarnings("removal")
@Override
public void execute(long timestamp, PeriodicType periodicType) {
AccessController.doPrivileged((PrivilegedAction<Void>) () -> {
execute();
return null;
}, controlContext);
}
private void execute() {
try {
getRunnable().run();
if (Logger.shouldLog(LogTag.JFR_EVENT, LogLevel.DEBUG)) {
Logger.log(LogTag.JFR_EVENT, LogLevel.DEBUG, "Executed periodic task for " + getEventType().getLogName());
}
} catch (Throwable t) {
// Prevent malicious user to propagate exception callback in the wrong context
Logger.log(LogTag.JFR_EVENT, LogLevel.WARN, "Exception occurred during execution of period task for " + getEventType().getLogName());
}
}
}