From 872a4ab695e11ecc7fb35772e3a4504b526c8963 Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Tue, 10 Oct 2023 15:43:27 +0200 Subject: [PATCH 1/9] Use common pool for analysis and compilation. --- .../pointsto/standalone/PointsToAnalyzer.java | 5 +- .../StandalonePointsToAnalysis.java | 10 +-- .../pointsto/AbstractAnalysisEngine.java | 5 +- .../graal/pointsto/PointsToAnalysis.java | 30 ++----- .../pointsto/heap/HeapSnapshotVerifier.java | 5 +- .../pointsto/util/CompletionExecutor.java | 60 ++----------- .../ReachabilityAnalysisEngine.java | 5 +- .../src/com/oracle/objectfile/ObjectFile.java | 3 +- .../graal/llvm/LLVMNativeImageCodeCache.java | 8 +- .../core/graal/llvm/LLVMToolchainUtils.java | 5 +- .../graal/llvm/objectfile/LLVMObjectFile.java | 5 +- .../com/oracle/svm/core/SubstrateOptions.java | 11 +++ .../com/oracle/svm/driver/NativeImage.java | 12 +-- .../svm/hosted/HostedConfiguration.java | 5 +- .../svm/hosted/NativeImageGenerator.java | 89 +++++++++---------- .../hosted/NativeImageGeneratorRunner.java | 26 +----- .../oracle/svm/hosted/NativeImageOptions.java | 42 +-------- .../oracle/svm/hosted/ProgressReporter.java | 6 +- .../analysis/NativeImagePointsToAnalysis.java | 6 +- ...NativeImageReachabilityAnalysisEngine.java | 19 ++-- .../oracle/svm/hosted/code/CompileQueue.java | 5 +- .../svm/hosted/image/AbstractImage.java | 31 +++---- .../hosted/image/LIRNativeImageCodeCache.java | 3 +- .../oracle/svm/hosted/image/NativeImage.java | 5 +- .../hosted/image/NativeImageCodeCache.java | 15 ++-- .../svm/hosted/image/NativeImageViaCC.java | 5 +- .../hosted/image/SharedLibraryImageViaCC.java | 5 +- 27 files changed, 140 insertions(+), 286 deletions(-) diff --git a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java index 5e5b4dad7517..8785e202b5cf 100644 --- a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java +++ b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java @@ -37,7 +37,6 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.debug.DebugContext; @@ -52,7 +51,6 @@ import com.oracle.graal.pointsto.AnalysisObjectScanningObserver; import com.oracle.graal.pointsto.AnalysisPolicy; -import com.oracle.graal.pointsto.PointsToAnalysis; import com.oracle.graal.pointsto.api.PointstoOptions; import com.oracle.graal.pointsto.flow.context.bytecode.BytecodeSensitiveAnalysisPolicy; import com.oracle.graal.pointsto.heap.HeapSnapshotVerifier; @@ -135,7 +133,6 @@ private PointsToAnalyzer(String mainEntryClass, OptionValues options) { SnippetReflectionProvider snippetReflection = originalProviders.getSnippetReflection(); MetaAccessProvider originalMetaAccess = originalProviders.getMetaAccess(); debugContext = new DebugContext.Builder(options, new GraalDebugHandlersFactory(snippetReflection)).build(); - ForkJoinPool executor = PointsToAnalysis.createExecutor(debugContext, Math.min(Runtime.getRuntime().availableProcessors(), 32)); StandaloneHost standaloneHost = new StandaloneHost(options, analysisClassLoader); int wordSize = getWordSize(); AnalysisPolicy analysisPolicy = PointstoOptions.AllocationSiteSensitiveHeap.getValue(options) ? new BytecodeSensitiveAnalysisPolicy(options) @@ -154,7 +151,7 @@ private PointsToAnalyzer(String mainEntryClass, OptionValues options) { originalProviders.getPlatformConfigurationProvider(), aMetaAccessExtensionProvider, originalProviders.getLoopsDataProvider()); standaloneHost.initializeProviders(aProviders); analysisName = getAnalysisName(mainEntryClass); - bigbang = new StandalonePointsToAnalysis(options, aUniverse, standaloneHost, aMetaAccess, snippetReflection, aConstantReflection, aProviders.getWordTypes(), executor, new TimerCollection()); + bigbang = new StandalonePointsToAnalysis(options, aUniverse, standaloneHost, aMetaAccess, snippetReflection, aConstantReflection, aProviders.getWordTypes(), new TimerCollection()); standaloneHost.setImageName(analysisName); aUniverse.setBigBang(bigbang); ImageHeap heap = new ImageHeap(); diff --git a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java index 439ada3ef91a..8e769b0bde7d 100644 --- a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java +++ b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.options.OptionValues; @@ -48,12 +47,9 @@ public class StandalonePointsToAnalysis extends PointsToAnalysis { private final Set addedClinits = ConcurrentHashMap.newKeySet(); - public StandalonePointsToAnalysis(OptionValues options, AnalysisUniverse universe, HostVM hostVM, - AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, - ForkJoinPool executorService, TimerCollection timerCollection) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, executorService, new UnsupportedFeatures(), timerCollection, - true); + public StandalonePointsToAnalysis(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, TimerCollection timerCollection) { + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new UnsupportedFeatures(), timerCollection, true); } @Override diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java index 5bad5d06986f..687995ec9550 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java @@ -27,7 +27,6 @@ import java.io.PrintWriter; import java.util.Collections; import java.util.List; -import java.util.concurrent.ForkJoinPool; import java.util.function.Function; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; @@ -101,7 +100,7 @@ public abstract class AbstractAnalysisEngine implements BigBang { @SuppressWarnings("this-escape") public AbstractAnalysisEngine(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, ForkJoinPool executorService, UnsupportedFeatures unsupportedFeatures, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection) { this.options = options; this.universe = universe; @@ -110,7 +109,7 @@ public AbstractAnalysisEngine(OptionValues options, AnalysisUniverse universe, H this.metaAccess = metaAccess; this.analysisPolicy = universe.analysisPolicy(); this.hostVM = hostVM; - this.executor = new CompletionExecutor(this, executorService); + this.executor = new CompletionExecutor(this); this.unsupportedFeatures = unsupportedFeatures; this.processFeaturesTimer = timerCollection.get(TimerCollection.Registry.FEATURES); diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java index 9b782a19ea96..5c6af4e84765 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java @@ -44,15 +44,6 @@ import java.util.function.Consumer; import java.util.stream.StreamSupport; -import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; -import jdk.graal.compiler.core.common.SuppressFBWarnings; -import jdk.graal.compiler.debug.DebugContext; -import jdk.graal.compiler.debug.Indent; -import jdk.graal.compiler.graph.Node; -import jdk.graal.compiler.graph.NodeList; -import jdk.graal.compiler.options.OptionValues; -import jdk.graal.compiler.word.WordTypes; - import com.oracle.graal.pointsto.api.HostVM; import com.oracle.graal.pointsto.api.PointstoOptions; import com.oracle.graal.pointsto.constraints.UnsupportedFeatures; @@ -86,6 +77,13 @@ import com.oracle.svm.util.ClassUtil; import com.oracle.svm.util.ImageGeneratorThreadMarker; +import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.debug.DebugContext; +import jdk.graal.compiler.debug.Indent; +import jdk.graal.compiler.graph.Node; +import jdk.graal.compiler.graph.NodeList; +import jdk.graal.compiler.options.OptionValues; +import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.meta.ConstantReflectionProvider; import jdk.vm.ci.meta.JavaKind; import jdk.vm.ci.meta.JavaType; @@ -111,9 +109,9 @@ public abstract class PointsToAnalysis extends AbstractAnalysisEngine { @SuppressWarnings("this-escape") public PointsToAnalysis(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, ForkJoinPool executorService, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, boolean strengthenGraalGraphs) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, executorService, unsupportedFeatures, timerCollection); + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection); this.typeFlowTimer = timerCollection.createTimer("(typeflow)"); this.strengthenGraalGraphs = strengthenGraalGraphs; @@ -537,16 +535,6 @@ public boolean doTypeflow() throws InterruptedException { return didSomeWork; } - @SuppressFBWarnings(value = "NP_NONNULL_PARAM_VIOLATION", justification = "ForkJoinPool does support null for the exception handler.") - public static ForkJoinPool createExecutor(DebugContext debug, int numberOfThreads) { - ForkJoinPool.ForkJoinWorkerThreadFactory factory = debugThreadFactory(debug.areScopesEnabled() || debug.areMetricsEnabled() ? debug : null); - return new ForkJoinPool(numberOfThreads, factory, null, false); - } - - private static ForkJoinPool.ForkJoinWorkerThreadFactory debugThreadFactory(DebugContext debug) { - return pool -> new SubstrateWorkerThread(pool, debug); - } - @Override public void onTypeInstantiated(AnalysisType type, AnalysisType.UsageKind usageKind) { /* Register the type as instantiated with all its super types. */ diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java index 13f22c94df56..e79f60ca0a6b 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java @@ -27,7 +27,6 @@ import static com.oracle.graal.pointsto.ObjectScanner.ScanReason; import java.util.Objects; -import java.util.concurrent.ForkJoinPool; import java.util.function.Consumer; import jdk.graal.compiler.options.Option; @@ -74,8 +73,8 @@ public HeapSnapshotVerifier(BigBang bb, ImageHeap imageHeap, ImageHeapScanner sc verbosity = Options.HeapVerifierVerbosity.getValue(bb.getOptions()); } - public boolean checkHeapSnapshot(UniverseMetaAccess metaAccess, ForkJoinPool threadPool, String stage) { - CompletionExecutor executor = new CompletionExecutor(bb, threadPool); + public boolean checkHeapSnapshot(UniverseMetaAccess metaAccess, String stage) { + CompletionExecutor executor = new CompletionExecutor(bb); executor.init(); return checkHeapSnapshot(metaAccess, executor, stage, false); } diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java index bfbecae57b10..6435a701c420 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java @@ -28,7 +28,6 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -64,7 +63,7 @@ private enum State { private List postedBeforeStart; private final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); - private ExecutorService executorService; + private final ForkJoinPool executorService; private final BigBang bb; private Timing timing; @@ -82,9 +81,9 @@ public interface Timing { void print(); } - public CompletionExecutor(BigBang bb, ForkJoinPool forkJoin) { + public CompletionExecutor(BigBang bb) { this.bb = bb; - executorService = forkJoin; + executorService = ForkJoinPool.commonPool(); state = new AtomicReference<>(State.UNUSED); postedOperations = new LongAdder(); completedOperations = new LongAdder(); @@ -146,18 +145,7 @@ public void execute(DebugContextRunnable command) { if (timing != null) { timing.addScheduled(command); } - - if (isSequential()) { - bb.getHostVM().recordActivity(); - try (DebugContext debug = command.getDebug(bb.getOptions(), bb.getDebugHandlerFactories()); - Scope s = debug.scope("Operation")) { - command.run(debug); - } - completedOperations.increment(); - } else { - executeService(command); - } - + executeService(command); break; default: throw JVMCIError.shouldNotReachHere(); @@ -165,9 +153,7 @@ public void execute(DebugContextRunnable command) { } private void executeService(DebugContextRunnable command) { - executorService.execute(() -> { - executeCommand(command); - }); + executorService.execute(() -> executeCommand(command)); } @SuppressWarnings("try") @@ -212,14 +198,6 @@ private void setState(State newState) { } public long complete() throws InterruptedException { - - if (isSequential()) { - long completed = completedOperations.sum(); - long posted = postedOperations.sum(); - assert completed == posted : completed + ", " + posted; - return posted; - } - long lastPrint = 0; if (timing != null) { timing.printHeader(); @@ -230,12 +208,7 @@ public long complete() throws InterruptedException { while (true) { assert state.get() == State.STARTED : state.get(); - boolean quiescent; - if (executorService instanceof ForkJoinPool) { - quiescent = ((ForkJoinPool) executorService).awaitQuiescence(100, TimeUnit.MILLISECONDS); - } else { - quiescent = executorService.awaitTermination(100, TimeUnit.MILLISECONDS); - } + boolean quiescent = executorService.awaitQuiescence(100, TimeUnit.MILLISECONDS); if (timing != null && !quiescent) { long curTime = System.nanoTime(); if (curTime - lastPrint > timing.getPrintIntervalNanos()) { @@ -265,12 +238,8 @@ public long getPostedOperations() { return postedOperations.sum() + (postedBeforeStart == null ? 0 : postedBeforeStart.size()); } - public boolean isSequential() { - return executorService == null; - } - public void shutdown() { - assert isSequential() || !(executorService instanceof ForkJoinPool) || !((ForkJoinPool) executorService).hasQueuedSubmissions() : "There should be no queued submissions on shutdown."; + assert !executorService.hasQueuedSubmissions() : "There should be no queued submissions on shutdown."; assert completedOperations.sum() == postedOperations.sum() : "Posted operations (" + postedOperations.sum() + ") must match completed (" + completedOperations.sum() + ") operations"; setState(State.UNUSED); } @@ -286,19 +255,4 @@ public boolean isStarted() { public State getState() { return state.get(); } - - public int parallelism() { - if (executorService instanceof ForkJoinPool) { - return ((ForkJoinPool) executorService).getParallelism(); - } - return 1; - } - - public ExecutorService getExecutorService() { - return executorService; - } - - public void setExecutorService(ExecutorService executorService) { - this.executorService = executorService; - } } diff --git a/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java b/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java index ea0912b41eef..f1eab31b64cc 100644 --- a/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java +++ b/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java @@ -30,7 +30,6 @@ import java.util.Deque; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.core.common.type.TypedConstant; @@ -80,9 +79,9 @@ public abstract class ReachabilityAnalysisEngine extends AbstractAnalysisEngine @SuppressWarnings("this-escape") public ReachabilityAnalysisEngine(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, ForkJoinPool executorService, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, executorService, unsupportedFeatures, timerCollection); + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection); this.executor.init(getTiming()); this.reachabilityTimer = timerCollection.createTimer("(reachability)"); diff --git a/substratevm/src/com.oracle.objectfile/src/com/oracle/objectfile/ObjectFile.java b/substratevm/src/com.oracle.objectfile/src/com/oracle/objectfile/ObjectFile.java index 519fa8734266..4f978d4b15d7 100644 --- a/substratevm/src/com.oracle.objectfile/src/com/oracle/objectfile/ObjectFile.java +++ b/substratevm/src/com.oracle.objectfile/src/com/oracle/objectfile/ObjectFile.java @@ -43,7 +43,6 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; -import java.util.concurrent.ForkJoinPool; import java.util.function.Consumer; import java.util.stream.StreamSupport; @@ -1268,7 +1267,7 @@ public Element getOffsetBootstrapElement() { private final Map> dependenciesByDependingElement = new IdentityHashMap<>(); private final Map> dependenciesByDependedOnElement = new IdentityHashMap<>(); - public void write(DebugContext context, Path outputFile, @SuppressWarnings("unused") ForkJoinPool forkJoinPool) throws IOException { + public void write(DebugContext context, Path outputFile) throws IOException { try (FileChannel channel = FileChannel.open(outputFile, StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.TRUNCATE_EXISTING, StandardOpenOption.CREATE)) { withDebugContext(context, "ObjectFile.write", () -> { write(channel); diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java index 0c8de6681173..6677d3c29752 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java @@ -42,10 +42,10 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import java.util.stream.IntStream; +import com.oracle.svm.core.SubstrateOptions; import org.graalvm.collections.Pair; import jdk.graal.compiler.code.CompilationResult; import jdk.graal.compiler.core.common.NumUtil; @@ -117,9 +117,9 @@ public int codeSizeFor(HostedMethod method) { @Override @SuppressWarnings({"unused", "try"}) - public void layoutMethods(DebugContext debug, BigBang bb, ForkJoinPool threadPool) { + public void layoutMethods(DebugContext debug, BigBang bb) { try (Indent indent = debug.logAndIndent("layout methods")) { - BatchExecutor executor = new BatchExecutor(bb, threadPool); + BatchExecutor executor = new BatchExecutor(bb); try (StopTimer t = TimerCollection.createTimerAndStart("(bitcode)")) { writeBitcode(executor); } @@ -153,7 +153,7 @@ private void writeBitcode(BatchExecutor executor) { private int createBitcodeBatches(BatchExecutor executor, DebugContext debug) { batchSize = LLVMOptions.LLVMMaxFunctionsPerBatch.getValue(); - int numThreads = executor.getExecutor().parallelism(); + int numThreads = SubstrateOptions.NumberOfThreads.getValue(); int idealSize = NumUtil.divideAndRoundUp(methodIndex.length, numThreads); if (idealSize < batchSize) { batchSize = idealSize; diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java index 15bb3af478f5..d7b746d668fe 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java @@ -29,7 +29,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ForkJoinPool; import java.util.function.Function; import java.util.function.IntFunction; @@ -170,8 +169,8 @@ public static void llvmCleanupStackMaps(DebugContext debug, String inputPath, Pa public static final class BatchExecutor { private CompletionExecutor executor; - public BatchExecutor(BigBang bb, ForkJoinPool threadPool) { - this.executor = new CompletionExecutor(bb, threadPool); + public BatchExecutor(BigBang bb) { + this.executor = new CompletionExecutor(bb); executor.init(); } diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java index 54a0e6b30775..bcce8a587fea 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -183,7 +182,7 @@ public SymbolTable getSymbolTable() { @Override @SuppressWarnings("try") - public final void write(DebugContext context, Path outputFile, ForkJoinPool forkJoinPool) throws IOException { + public final void write(DebugContext context, Path outputFile) throws IOException { List sortedObjectFileElements = new ArrayList<>(); bake(sortedObjectFileElements); @@ -192,7 +191,7 @@ public final void write(DebugContext context, Path outputFile, ForkJoinPool fork writeParts(); - BatchExecutor batchExecutor = new BatchExecutor(bb, forkJoinPool); + BatchExecutor batchExecutor = new BatchExecutor(bb); compileBitcodeBatches(batchExecutor, context, numBatches); diff --git a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java index 2948aacaa928..03788b94e10a 100644 --- a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java +++ b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java @@ -58,6 +58,7 @@ import com.oracle.svm.core.option.SubstrateOptionsParser; import com.oracle.svm.core.thread.VMOperationControl; import com.oracle.svm.core.util.UserError; +import com.oracle.svm.core.util.VMError; import com.oracle.svm.util.LogUtils; import com.oracle.svm.util.ModuleSupport; import com.oracle.svm.util.ReflectionUtil; @@ -106,6 +107,16 @@ public static boolean parseOnce() { @Option(help = "Name of the output file to be generated", type = OptionType.User)// public static final HostedOptionKey Name = new HostedOptionKey<>(""); + /** + * Configures the number of threads of the common pool (see driver). + */ + @APIOption(name = "parallelism")// + @Option(help = "The maximum number of threads to use concurrently during native image generation.")// + public static final HostedOptionKey NumberOfThreads = new HostedOptionKey<>(Math.max(1, Math.min(Runtime.getRuntime().availableProcessors(), 32)), key -> { + int numberOfThreads = key.getValue(); + VMError.guarantee(numberOfThreads >= 1, "Number of threads must be at least 1. Validation should have happened in driver."); + }); + @APIOption(name = "shared")// @Option(help = "Build shared library")// public static final HostedOptionKey SharedLibrary = new HostedOptionKey<>(false); diff --git a/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java b/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java index ca573188bb4e..5641e7ecf1a6 100644 --- a/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java +++ b/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java @@ -97,7 +97,6 @@ import com.oracle.svm.driver.metainf.NativeImageMetaInfResourceProcessor; import com.oracle.svm.driver.metainf.NativeImageMetaInfWalker; import com.oracle.svm.hosted.NativeImageGeneratorRunner; -import com.oracle.svm.hosted.NativeImageOptions; import com.oracle.svm.hosted.NativeImageSystemClassLoader; import com.oracle.svm.util.LogUtils; import com.oracle.svm.util.ModuleSupport; @@ -269,7 +268,7 @@ private static String oR(OptionKey option) { final String oHInspectServerContentPath = oH(PointstoOptions.InspectServerContentPath); final String oHDeadlockWatchdogInterval = oH(SubstrateOptions.DeadlockWatchdogInterval); - static final String oHNumberOfThreads = oH(NativeImageOptions.NumberOfThreads); + static final String oHNumberOfThreads = oH(SubstrateOptions.NumberOfThreads); final Map imageBuilderEnvironment = new HashMap<>(); private final ArrayList imageBuilderArgs = new ArrayList<>(); @@ -1098,14 +1097,15 @@ private int completeImageBuild() { Integer maxNumberOfThreads = getMaxNumberOfThreads(); if (maxNumberOfThreads != null) { - if (maxNumberOfThreads >= 2) { + if (maxNumberOfThreads >= 1) { /* - * Set number of threads in common pool. Subtract one because the main thread helps - * to process tasks. + * maxNumberOfThreads - 1 because the main thread always helps to process tasks. In + * single-threaded mode (parallelism=0 for common pool), only the main thread + * processes tasks. */ imageBuilderJavaArgs.add("-Djava.util.concurrent.ForkJoinPool.common.parallelism=" + (maxNumberOfThreads - 1)); } else { - throw showError("The number of threads was set to " + maxNumberOfThreads + ". Please set the '--parallelism' option to at least 2."); + throw showError("The number of threads was set to " + maxNumberOfThreads + ". Please set the '--parallelism' option to at least 1."); } } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java index 0465a396fe21..e7255096bfbf 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java @@ -30,7 +30,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.core.common.CompressEncoding; @@ -174,9 +173,9 @@ public SVMHost createHostVM(OptionValues options, ClassLoader classLoader, Class } public CompileQueue createCompileQueue(DebugContext debug, FeatureHandler featureHandler, HostedUniverse hostedUniverse, - RuntimeConfiguration runtimeConfiguration, boolean deoptimizeAll, SnippetReflectionProvider aSnippetReflection, ForkJoinPool executor) { + RuntimeConfiguration runtimeConfiguration, boolean deoptimizeAll, SnippetReflectionProvider aSnippetReflection) { - return new CompileQueue(debug, featureHandler, hostedUniverse, runtimeConfiguration, deoptimizeAll, aSnippetReflection, executor); + return new CompileQueue(debug, featureHandler, hostedUniverse, runtimeConfiguration, deoptimizeAll, aSnippetReflection); } public MethodTypeFlowBuilder createMethodTypeFlowBuilder(PointsToAnalysis bb, PointsToAnalysisMethod method, MethodFlowsGraph flowsGraph, MethodFlowsGraph.GraphKind graphKind) { diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java index c159659fdfc2..1714fd92b78b 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java @@ -53,7 +53,6 @@ import java.util.Map; import java.util.ServiceLoader; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BooleanSupplier; import java.util.stream.Stream; @@ -519,46 +518,40 @@ public void run(Map entryPoints, JavaMainSupport javaMainSupport, String imageName, NativeImageKind k, SubstitutionProcessor harnessSubstitutions, - ForkJoinPool compilationExecutor, ForkJoinPool analysisExecutor, EconomicSet allOptionNames, TimerCollection timerCollection) { - try { - if (!buildStarted.compareAndSet(false, true)) { - throw UserError.abort("An image build has already been performed with this generator."); - } + if (!buildStarted.compareAndSet(false, true)) { + throw UserError.abort("An image build has already been performed with this generator."); + } - try { - /* - * JVMCI 20.2-b01 introduced new methods for linking and querying whether an - * interface has default methods. Fail early if these methods are missing. - */ - ResolvedJavaType.class.getDeclaredMethod("link"); - } catch (ReflectiveOperationException ex) { - throw UserError.abort("JVMCI version provided %s is missing the 'ResolvedJavaType.link()' method added in jvmci-20.2-b01. " + - "Please use the latest JVMCI JDK from %s.", System.getProperty("java.home"), OPEN_LABSJDK_RELEASE_URL_PATTERN); - } + try { + /* + * JVMCI 20.2-b01 introduced new methods for linking and querying whether an interface + * has default methods. Fail early if these methods are missing. + */ + ResolvedJavaType.class.getDeclaredMethod("link"); + } catch (ReflectiveOperationException ex) { + throw UserError.abort("JVMCI version provided %s is missing the 'ResolvedJavaType.link()' method added in jvmci-20.2-b01. " + + "Please use the latest JVMCI JDK from %s.", System.getProperty("java.home"), OPEN_LABSJDK_RELEASE_URL_PATTERN); + } - setSystemPropertiesForImageLate(k); + setSystemPropertiesForImageLate(k); - ImageSingletonsSupportImpl.HostedManagement.install(new ImageSingletonsSupportImpl.HostedManagement()); + ImageSingletonsSupportImpl.HostedManagement.install(new ImageSingletonsSupportImpl.HostedManagement()); - ImageSingletons.add(ProgressReporter.class, reporter); - ImageSingletons.add(DeadlockWatchdog.class, loader.watchdog); - ImageSingletons.add(TimerCollection.class, timerCollection); - ImageSingletons.add(ImageBuildStatistics.TimerCollectionPrinter.class, timerCollection); - ImageSingletons.add(AnnotationExtractor.class, loader.classLoaderSupport.annotationExtractor); - ImageSingletons.add(BuildArtifacts.class, (type, artifact) -> buildArtifacts.computeIfAbsent(type, t -> new ArrayList<>()).add(artifact)); - ImageSingletons.add(HostedOptionValues.class, new HostedOptionValues(optionProvider.getHostedValues())); - ImageSingletons.add(RuntimeOptionValues.class, new RuntimeOptionValues(optionProvider.getRuntimeValues(), allOptionNames)); + ImageSingletons.add(ProgressReporter.class, reporter); + ImageSingletons.add(DeadlockWatchdog.class, loader.watchdog); + ImageSingletons.add(TimerCollection.class, timerCollection); + ImageSingletons.add(ImageBuildStatistics.TimerCollectionPrinter.class, timerCollection); + ImageSingletons.add(AnnotationExtractor.class, loader.classLoaderSupport.annotationExtractor); + ImageSingletons.add(BuildArtifacts.class, (type, artifact) -> buildArtifacts.computeIfAbsent(type, t -> new ArrayList<>()).add(artifact)); + ImageSingletons.add(HostedOptionValues.class, new HostedOptionValues(optionProvider.getHostedValues())); + ImageSingletons.add(RuntimeOptionValues.class, new RuntimeOptionValues(optionProvider.getRuntimeValues(), allOptionNames)); - try (TemporaryBuildDirectoryProviderImpl tempDirectoryProvider = new TemporaryBuildDirectoryProviderImpl()) { - ImageSingletons.add(TemporaryBuildDirectoryProvider.class, tempDirectoryProvider); - doRun(entryPoints, javaMainSupport, imageName, k, harnessSubstitutions, compilationExecutor, analysisExecutor); - } finally { - reporter.ensureCreationStageEndCompleted(); - } + try (TemporaryBuildDirectoryProviderImpl tempDirectoryProvider = new TemporaryBuildDirectoryProviderImpl()) { + ImageSingletons.add(TemporaryBuildDirectoryProvider.class, tempDirectoryProvider); + doRun(entryPoints, javaMainSupport, imageName, k, harnessSubstitutions); } finally { - analysisExecutor.shutdownNow(); - compilationExecutor.shutdownNow(); + reporter.ensureCreationStageEndCompleted(); } } @@ -583,8 +576,7 @@ public static void clearSystemPropertiesForImage() { @SuppressWarnings("try") protected void doRun(Map entryPoints, JavaMainSupport javaMainSupport, String imageName, NativeImageKind k, - SubstitutionProcessor harnessSubstitutions, - ForkJoinPool compilationExecutor, ForkJoinPool analysisExecutor) { + SubstitutionProcessor harnessSubstitutions) { List hostedEntryPoints = new ArrayList<>(); OptionValues options = HostedOptionValues.singleton(); @@ -592,7 +584,7 @@ protected void doRun(Map entryPoints, try (DebugContext debug = new Builder(options, new GraalDebugHandlersFactory(originalSnippetReflection)).build(); DebugCloseable featureCleanup = () -> featureHandler.forEachFeature(Feature::cleanup)) { - setupNativeImage(options, entryPoints, javaMainSupport, harnessSubstitutions, analysisExecutor, originalSnippetReflection, debug); + setupNativeImage(options, entryPoints, javaMainSupport, harnessSubstitutions, originalSnippetReflection, debug); boolean returnAfterAnalysis = runPointsToAnalysis(imageName, options, debug); if (returnAfterAnalysis) { @@ -670,8 +662,7 @@ protected void doRun(Map entryPoints, NativeImageCodeCache codeCache; CompileQueue compileQueue; try (StopTimer t = TimerCollection.createTimerAndStart(TimerCollection.Registry.COMPILE_TOTAL)) { - compileQueue = HostedConfiguration.instance().createCompileQueue(debug, featureHandler, hUniverse, runtimeConfiguration, DeoptTester.enabled(), - bb.getSnippetReflectionProvider(), compilationExecutor); + compileQueue = HostedConfiguration.instance().createCompileQueue(debug, featureHandler, hUniverse, runtimeConfiguration, DeoptTester.enabled(), bb.getSnippetReflectionProvider()); if (ImageSingletons.contains(RuntimeCompilationSupport.class)) { ImageSingletons.lookup(RuntimeCompilationSupport.class).onCompileQueueCreation(bb, hUniverse, compileQueue); } @@ -684,8 +675,8 @@ protected void doRun(Map entryPoints, codeCache = NativeImageCodeCacheFactory.get().newCodeCache(compileQueue, heap, loader.platform, ImageSingletons.lookup(TemporaryBuildDirectoryProvider.class).getTemporaryBuildDirectory()); codeCache.layoutConstants(); - codeCache.layoutMethods(debug, bb, compilationExecutor); - codeCache.buildRuntimeMetadata(bb.getSnippetReflectionProvider(), compilationExecutor); + codeCache.layoutMethods(debug, bb); + codeCache.buildRuntimeMetadata(bb.getSnippetReflectionProvider()); } AfterCompilationAccessImpl config = new AfterCompilationAccessImpl(featureHandler, loader, aUniverse, hUniverse, compileQueue.getCompilations(), codeCache, heap, debug, @@ -695,7 +686,7 @@ protected void doRun(Map entryPoints, } /* Re-run shadow heap verification after compilation. */ - aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, compilationExecutor, "after compilation"); + aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, "after compilation"); CodeCacheProvider codeCacheProvider = runtimeConfiguration.getBackendForNormalMethod().getProviders().getCodeCache(); reporter.printCreationStart(); @@ -714,7 +705,7 @@ protected void doRun(Map entryPoints, featureHandler.forEachFeature(feature -> feature.afterHeapLayout(config)); /* Re-run shadow heap verification after heap layout. */ - aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, compilationExecutor, "after heap layout"); + aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, "after heap layout"); createAbstractImage(k, hostedEntryPoints, heap, hMetaAccess, codeCache); @@ -755,7 +746,7 @@ protected void doRun(Map entryPoints, * not is an implementation detail of the image. */ Path tmpDir = ImageSingletons.lookup(TemporaryBuildDirectoryProvider.class).getTemporaryBuildDirectory(); - LinkerInvocation inv = image.write(debug, generatedFiles(HostedOptionValues.singleton()), tmpDir, imageName, beforeConfig, compilationExecutor); + LinkerInvocation inv = image.write(debug, generatedFiles(HostedOptionValues.singleton()), tmpDir, imageName, beforeConfig); if (NativeImageOptions.ExitAfterRelocatableImageWrite.getValue()) { return; } @@ -869,7 +860,7 @@ protected boolean verifyAssignableTypes() { @SuppressWarnings("try") protected void setupNativeImage(OptionValues options, Map entryPoints, JavaMainSupport javaMainSupport, - SubstitutionProcessor harnessSubstitutions, ForkJoinPool analysisExecutor, SnippetReflectionProvider originalSnippetReflection, DebugContext debug) { + SubstitutionProcessor harnessSubstitutions, SnippetReflectionProvider originalSnippetReflection, DebugContext debug) { try (Indent ignored = debug.logAndIndent("setup native-image builder")) { try (StopTimer ignored1 = TimerCollection.createTimerAndStart(TimerCollection.Registry.SETUP)) { SubstrateTargetDescription target = createTarget(); @@ -934,7 +925,7 @@ protected void setupNativeImage(OptionValues options, Map NumberOfThreads = new HostedOptionKey<>(Math.max(2, Math.min(Runtime.getRuntime().availableProcessors(), 32)), key -> { - int numberOfThreads = key.getValue(); - VMError.guarantee(numberOfThreads >= 2, "Number of threads must be at least 2. Validation should have happened in driver."); - }); - - /* - * Analysis scales well up to 12 cores and gives slight improvements until 18 cores. We set the - * default value to 16 to minimize wasted resources in large machines. - */ - @Option(help = "The number of threads to use for analysis during native image generation. The number must be smaller than the NumberOfThreads.", deprecated = true, deprecationMessage = "Please use '--parallelism' instead.")// + @Option(help = "Deprecated, option no longer has any effect", deprecated = true, deprecationMessage = "Please use '--parallelism' instead.")// public static final HostedOptionKey NumberOfAnalysisThreads = new HostedOptionKey<>(-1); @Option(help = "Return after analysis")// @@ -214,13 +198,13 @@ public static CStandards getCStandard() { @Option(help = "Exit after writing relocatable file")// public static final HostedOptionKey ExitAfterRelocatableImageWrite = new HostedOptionKey<>(false); - @Option(help = "Throw unsafe operation offset errors.)")// + @Option(help = "Throw unsafe operation offset errors.")// public static final HostedOptionKey ThrowUnsafeOffsetErrors = new HostedOptionKey<>(true); - @Option(help = "Print unsafe operation offset warnings.)")// + @Option(help = "Print unsafe operation offset warnings.")// public static final HostedOptionKey ReportUnsafeOffsetWarnings = new HostedOptionKey<>(false); - @Option(help = "Print unsafe operation offset warnings.)")// + @Option(help = "Print unsafe operation offset warnings.")// public static final HostedOptionKey UnsafeOffsetWarningsAreFatal = new HostedOptionKey<>(false); /** @@ -272,22 +256,4 @@ protected void onValueUpdate(EconomicMap, Object> values, Boolean o } } }; - - public static int getNumberOfAnalysisThreads(int maxNumberOfThreads, OptionValues optionValues) { - int analysisThreads; - if (NumberOfAnalysisThreads.hasBeenSet(optionValues)) { - analysisThreads = NumberOfAnalysisThreads.getValue(optionValues); - } else { - analysisThreads = Math.min(maxNumberOfThreads, DEFAULT_MAX_ANALYSIS_SCALING); - } - if (analysisThreads < 2) { - throw UserError.abort("Number of analysis threads was set to " + analysisThreads + ". Please set the '-H:NumberOfAnalysisThreads' option to at least 2."); - } - if (analysisThreads > maxNumberOfThreads) { - throw UserError.abort( - "NumberOfAnalysisThreads is not allowed to be larger than the number of threads set with the '--parallelism' option. Please set the '-H:NumberOfAnalysisThreads' option to a value between 1 and " + - (maxNumberOfThreads + 1) + "."); - } - return analysisThreads; - } } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java index 64dceb12a375..309031cd0a71 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java @@ -392,13 +392,13 @@ private void printResourceInfo() { maxHeapSuffix = "set via '%s'".formatted(xmxValueOrNull); } - int maxNumberOfThreads = NativeImageOptions.NumberOfThreads.getValue(); + int maxNumberOfThreads = SubstrateOptions.NumberOfThreads.getValue(); recordJsonMetric(ResourceUsageKey.PARALLELISM, maxNumberOfThreads); int availableProcessors = runtime.availableProcessors(); recordJsonMetric(ResourceUsageKey.CPU_CORES_TOTAL, availableProcessors); String maxNumberOfThreadsSuffix = "determined at start"; - if (NativeImageOptions.NumberOfThreads.hasBeenSet()) { - maxNumberOfThreadsSuffix = "set via '%s'".formatted(SubstrateOptionsParser.commandArgument(NativeImageOptions.NumberOfThreads, Integer.toString(maxNumberOfThreads))); + if (SubstrateOptions.NumberOfThreads.hasBeenSet()) { + maxNumberOfThreadsSuffix = "set via '%s'".formatted(SubstrateOptionsParser.commandArgument(SubstrateOptions.NumberOfThreads, Integer.toString(maxNumberOfThreads))); } l().printLineSeparator(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java index f096d25329d0..6411ed167001 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java @@ -27,7 +27,6 @@ import java.lang.reflect.Executable; import java.lang.reflect.Method; import java.lang.reflect.Modifier; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.options.OptionValues; @@ -66,10 +65,9 @@ public class NativeImagePointsToAnalysis extends PointsToAnalysis implements Inf public NativeImagePointsToAnalysis(OptionValues options, AnalysisUniverse universe, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, - AnnotationSubstitutionProcessor annotationSubstitutionProcessor, ForkJoinPool executor, UnsupportedFeatures unsupportedFeatures, + AnnotationSubstitutionProcessor annotationSubstitutionProcessor, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection) { - super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, executor, unsupportedFeatures, timerCollection, - SubstrateOptions.parseOnce()); + super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection, SubstrateOptions.parseOnce()); this.annotationSubstitutionProcessor = annotationSubstitutionProcessor; dynamicHubInitializer = new DynamicHubInitializer(this); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java index 0b129a9cf1c8..0330bd762c8e 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java @@ -24,12 +24,6 @@ */ package com.oracle.svm.hosted.analysis; -import java.util.concurrent.ForkJoinPool; - -import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; -import jdk.graal.compiler.options.OptionValues; -import jdk.graal.compiler.word.WordTypes; - import com.oracle.graal.pointsto.meta.AnalysisField; import com.oracle.graal.pointsto.meta.AnalysisMetaAccess; import com.oracle.graal.pointsto.meta.AnalysisType; @@ -41,6 +35,9 @@ import com.oracle.svm.hosted.SVMHost; import com.oracle.svm.hosted.substitute.AnnotationSubstitutionProcessor; +import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.options.OptionValues; +import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.meta.ConstantReflectionProvider; public class NativeImageReachabilityAnalysisEngine extends ReachabilityAnalysisEngine implements Inflation { @@ -51,12 +48,10 @@ public class NativeImageReachabilityAnalysisEngine extends ReachabilityAnalysisE private final CustomTypeFieldHandler unknownFieldHandler; @SuppressWarnings("this-escape") - public NativeImageReachabilityAnalysisEngine(OptionValues options, AnalysisUniverse universe, - AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, - AnnotationSubstitutionProcessor annotationSubstitutionProcessor, - ForkJoinPool executor, TimerCollection timerCollection, ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { - super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, executor, new SubstrateUnsupportedFeatures(), timerCollection, + public NativeImageReachabilityAnalysisEngine(OptionValues options, AnalysisUniverse universe, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, AnnotationSubstitutionProcessor annotationSubstitutionProcessor, TimerCollection timerCollection, + ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { + super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new SubstrateUnsupportedFeatures(), timerCollection, reachabilityMethodProcessingHandler); this.annotationSubstitutionProcessor = annotationSubstitutionProcessor; this.strengthenGraalGraphs = SubstrateOptions.parseOnce(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java index b7592f7e9287..71a3028aca69 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java @@ -37,7 +37,6 @@ import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ForkJoinPool; import org.graalvm.collections.EconomicMap; import jdk.graal.compiler.api.replacements.Fold; @@ -361,14 +360,14 @@ public Description getDescription() { @SuppressWarnings("this-escape") public CompileQueue(DebugContext debug, FeatureHandler featureHandler, HostedUniverse universe, RuntimeConfiguration runtimeConfiguration, Boolean deoptimizeAll, - SnippetReflectionProvider snippetReflection, ForkJoinPool executorService) { + SnippetReflectionProvider snippetReflection) { this.universe = universe; this.compilations = new ConcurrentHashMap<>(); this.runtimeConfig = runtimeConfiguration; this.metaAccess = runtimeConfiguration.getProviders().getMetaAccess(); this.deoptimizeAll = deoptimizeAll; this.dataCache = new ConcurrentHashMap<>(); - this.executor = new CompletionExecutor(universe.getBigBang(), executorService); + this.executor = new CompletionExecutor(universe.getBigBang()); this.featureHandler = featureHandler; this.snippetReflection = snippetReflection; this.graphTransplanter = createGraphTransplanter(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/AbstractImage.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/AbstractImage.java index 8efb24238a98..1937ff5c7081 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/AbstractImage.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/AbstractImage.java @@ -26,7 +26,6 @@ import java.nio.file.Path; import java.util.List; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.debug.DebugContext; @@ -54,16 +53,12 @@ public enum NativeImageKind { SHARED_LIBRARY(false) { @Override public String getFilenameSuffix() { - switch (ObjectFile.getNativeFormat()) { - case ELF: - return ".so"; - case MACH_O: - return ".dylib"; - case PECOFF: - return ".dll"; - default: - throw new AssertionError("Unreachable"); - } + return switch (ObjectFile.getNativeFormat()) { + case ELF -> ".so"; + case MACH_O -> ".dylib"; + case PECOFF -> ".dll"; + default -> throw new AssertionError("Unreachable"); + }; } }, EXECUTABLE(true), @@ -123,17 +118,17 @@ public NativeLibraries getNativeLibs() { /** * Write the image to the named file. */ - public abstract LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config, ForkJoinPool forkJoinPool); + public abstract LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config); // factory method public static AbstractImage create(NativeImageKind k, HostedUniverse universe, HostedMetaAccess metaAccess, NativeLibraries nativeLibs, NativeImageHeap heap, NativeImageCodeCache codeCache, List entryPoints, ClassLoader classLoader) { - switch (k) { - case SHARED_LIBRARY: - return new SharedLibraryImageViaCC(universe, metaAccess, nativeLibs, heap, codeCache, entryPoints, classLoader); - default: - return new ExecutableImageViaCC(k, universe, metaAccess, nativeLibs, heap, codeCache, entryPoints, classLoader); - } + return switch (k) { + case SHARED_LIBRARY -> + new SharedLibraryImageViaCC(universe, metaAccess, nativeLibs, heap, codeCache, entryPoints, classLoader); + case EXECUTABLE, STATIC_EXECUTABLE -> + new ExecutableImageViaCC(k, universe, metaAccess, nativeLibs, heap, codeCache, entryPoints, classLoader); + }; } public abstract String[] makeLaunchCommand(AbstractImage.NativeImageKind k, String imageName, Path binPath, Path workPath, java.lang.reflect.Method method); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/LIRNativeImageCodeCache.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/LIRNativeImageCodeCache.java index efa5b82136b4..c7cf1c37bec5 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/LIRNativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/LIRNativeImageCodeCache.java @@ -30,7 +30,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -139,7 +138,7 @@ private boolean verifyMethodLayout() { @SuppressWarnings("try") @Override - public void layoutMethods(DebugContext debug, BigBang bb, ForkJoinPool threadPool) { + public void layoutMethods(DebugContext debug, BigBang bb) { try (Indent indent = debug.logAndIndent("layout methods")) { // Assign initial location to all methods. diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImage.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImage.java index 68bb406bb11a..90d4f5b6b094 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImage.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImage.java @@ -46,7 +46,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import org.graalvm.collections.Pair; @@ -157,13 +156,13 @@ public NativeImage(NativeImageKind k, HostedUniverse universe, HostedMetaAccess @Override public abstract String[] makeLaunchCommand(NativeImageKind k, String imageName, Path binPath, Path workPath, java.lang.reflect.Method method); - protected final void write(DebugContext context, Path outputFile, ForkJoinPool forkJoinPool) { + protected final void write(DebugContext context, Path outputFile) { try { Path outFileParent = outputFile.normalize().getParent(); if (outFileParent != null) { Files.createDirectories(outFileParent); } - objectFile.write(context, outputFile, forkJoinPool); + objectFile.write(context, outputFile); } catch (Exception ex) { throw shouldNotReachHere(ex); } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java index 195f666b8952..ece30a9b5148 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java @@ -46,7 +46,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.ForkJoinPool; import java.util.stream.Collectors; import com.oracle.svm.hosted.DeadlockWatchdog; @@ -191,7 +190,7 @@ protected CompilationResult compilationResultFor(HostedMethod method) { return compilations.get(method); } - public abstract void layoutMethods(DebugContext debug, BigBang bb, ForkJoinPool threadPool); + public abstract void layoutMethods(DebugContext debug, BigBang bb); public void layoutConstants() { for (Pair pair : getOrderedCompilations()) { @@ -263,11 +262,11 @@ public int getAlignedConstantsSize() { return ConfigurationValues.getObjectLayout().alignUp(getConstantsSize()); } - public void buildRuntimeMetadata(SnippetReflectionProvider snippetReflectionProvider, ForkJoinPool threadPool) { - buildRuntimeMetadata(snippetReflectionProvider, threadPool, new MethodPointer(getFirstCompilation().getLeft(), true), WordFactory.signed(getCodeAreaSize())); + public void buildRuntimeMetadata(SnippetReflectionProvider snippetReflectionProvider) { + buildRuntimeMetadata(snippetReflectionProvider, new MethodPointer(getFirstCompilation().getLeft(), true), WordFactory.signed(getCodeAreaSize())); } - protected void buildRuntimeMetadata(SnippetReflectionProvider snippetReflection, ForkJoinPool threadPool, CFunctionPointer firstMethod, UnsignedWord codeSize) { + protected void buildRuntimeMetadata(SnippetReflectionProvider snippetReflection, CFunctionPointer firstMethod, UnsignedWord codeSize) { // Build run-time metadata. HostedFrameInfoCustomization frameInfoCustomization = new HostedFrameInfoCustomization(); CodeInfoEncoder.Encoders encoders = new CodeInfoEncoder.Encoders(); @@ -425,7 +424,7 @@ protected void buildRuntimeMetadata(SnippetReflectionProvider snippetReflection, verifyDeoptEntries(imageCodeInfo); } - assert verifyMethods(hUniverse, threadPool, codeInfoEncoder, imageCodeInfo); + assert verifyMethods(hUniverse, codeInfoEncoder, imageCodeInfo); } protected HostedImageCodeInfo installCodeInfo(SnippetReflectionProvider snippetReflection, CFunctionPointer firstMethod, UnsignedWord codeSize, CodeInfoEncoder codeInfoEncoder, @@ -568,12 +567,12 @@ private static boolean error(HostedMethod method, long encodedBci, String msg) { return true; } - protected boolean verifyMethods(HostedUniverse hUniverse, ForkJoinPool threadPool, CodeInfoEncoder codeInfoEncoder, CodeInfo codeInfo) { + protected boolean verifyMethods(HostedUniverse hUniverse, CodeInfoEncoder codeInfoEncoder, CodeInfo codeInfo) { /* * Run method verification in parallel to reduce computation time. */ BigBang bb = hUniverse.getBigBang(); - CompletionExecutor executor = new CompletionExecutor(bb, threadPool); + CompletionExecutor executor = new CompletionExecutor(bb); try { executor.init(); executor.start(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageViaCC.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageViaCC.java index 7870870bc43f..cce7224184ac 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageViaCC.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageViaCC.java @@ -34,7 +34,6 @@ import java.util.Collections; import java.util.Formatter; import java.util.List; -import java.util.concurrent.ForkJoinPool; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -89,13 +88,13 @@ private static List diagnoseLinkerFailure(String linkerOutput) { @Override @SuppressWarnings("try") - public LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config, ForkJoinPool forkJoinPool) { + public LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config) { try (Indent indent = debug.logAndIndent("Writing native image")) { // 0. Free codecache to make space for writing the objectFile codeCache.purge(); // 1. write the relocatable file - write(debug, tempDirectory.resolve(imageName + ObjectFile.getFilenameSuffix()), forkJoinPool); + write(debug, tempDirectory.resolve(imageName + ObjectFile.getFilenameSuffix())); if (NativeImageOptions.ExitAfterRelocatableImageWrite.getValue()) { return null; } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/SharedLibraryImageViaCC.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/SharedLibraryImageViaCC.java index 44e18ba9c9c0..2f4d73adc14f 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/SharedLibraryImageViaCC.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/SharedLibraryImageViaCC.java @@ -28,7 +28,6 @@ import java.lang.reflect.Method; import java.nio.file.Path; import java.util.List; -import java.util.concurrent.ForkJoinPool; import jdk.graal.compiler.debug.DebugContext; @@ -53,8 +52,8 @@ public String[] makeLaunchCommand(NativeImageKind k, String imageName, Path binP } @Override - public LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config, ForkJoinPool forkJoinPool) { - LinkerInvocation inv = super.write(debug, outputDirectory, tempDirectory, imageName, config, forkJoinPool); + public LinkerInvocation write(DebugContext debug, Path outputDirectory, Path tempDirectory, String imageName, BeforeImageWriteAccessImpl config) { + LinkerInvocation inv = super.write(debug, outputDirectory, tempDirectory, imageName, config); writeHeaderFiles(outputDirectory, imageName, false); writeHeaderFiles(outputDirectory, imageName, true); return inv; From e9a1f2da51c3471b637f4616cb88fa875bfd384c Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Wed, 18 Oct 2023 08:54:29 +0200 Subject: [PATCH 2/9] Load classes using the common pool. --- .../com/oracle/svm/hosted/ImageClassLoader.java | 14 ++------------ .../svm/hosted/NativeImageClassLoaderSupport.java | 1 + 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java index d861bcca5010..70783a3d6761 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java @@ -57,7 +57,6 @@ public final class ImageClassLoader { * This cannot be a HostedOption because the option parsing already relies on the list of loaded * classes. */ - private static final int CLASS_LOADING_MAX_SCALING = 8; private static final int CLASS_LOADING_TIMEOUT_IN_MINUTES = 10; static { @@ -87,20 +86,11 @@ public final class ImageClassLoader { } public void loadAllClasses() throws InterruptedException { - ForkJoinPool executor = new ForkJoinPool(Math.min(Runtime.getRuntime().availableProcessors(), CLASS_LOADING_MAX_SCALING)) { - @Override - public void execute(Runnable task) { - super.execute(() -> { - task.run(); - watchdog.recordActivity(); - }); - } - }; + ForkJoinPool executor = ForkJoinPool.commonPool(); try { classLoaderSupport.loadAllClasses(executor, this); } finally { - executor.shutdown(); - executor.awaitTermination(CLASS_LOADING_TIMEOUT_IN_MINUTES, TimeUnit.MINUTES); + executor.awaitQuiescence(CLASS_LOADING_TIMEOUT_IN_MINUTES, TimeUnit.MINUTES); } classLoaderSupport.reportBuilderClassesInApplication(); } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageClassLoaderSupport.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageClassLoaderSupport.java index be6d69b18ad1..9822516daf8c 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageClassLoaderSupport.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageClassLoaderSupport.java @@ -879,6 +879,7 @@ private void handleClassFileName(URI container, Module module, String className, } imageClassLoader.handleClass(clazz); } + imageClassLoader.watchdog.recordActivity(); } } From c58619b8c0338b00a062fa8b6795e178a6752553 Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Thu, 19 Oct 2023 09:16:00 +0200 Subject: [PATCH 3/9] Await completion of class loading tasks. --- .../com/oracle/svm/hosted/ImageClassLoader.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java index 70783a3d6761..624b129bf195 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ImageClassLoader.java @@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import com.oracle.svm.util.LogUtils; import org.graalvm.collections.EconomicSet; import jdk.graal.compiler.debug.GraalError; import jdk.graal.compiler.word.Word; @@ -53,12 +54,6 @@ public final class ImageClassLoader { - /* - * This cannot be a HostedOption because the option parsing already relies on the list of loaded - * classes. - */ - private static final int CLASS_LOADING_TIMEOUT_IN_MINUTES = 10; - static { /* * ImageClassLoader is one of the first classes used during image generation, so early @@ -90,7 +85,14 @@ public void loadAllClasses() throws InterruptedException { try { classLoaderSupport.loadAllClasses(executor, this); } finally { - executor.awaitQuiescence(CLASS_LOADING_TIMEOUT_IN_MINUTES, TimeUnit.MINUTES); + boolean isQuiescence = false; + while (!isQuiescence) { + isQuiescence = executor.awaitQuiescence(10, TimeUnit.MINUTES); + if (!isQuiescence) { + LogUtils.warning("Class loading is slow. Waiting for tasks to complete..."); + /* DeadlockWatchdog should fail the build eventually. */ + } + } } classLoaderSupport.reportBuilderClassesInApplication(); } From a0ada6eb4c69c43ac2d9918fa3dfb784b1662977 Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Thu, 19 Oct 2023 12:54:18 +0200 Subject: [PATCH 4/9] Extract reportFailureState logic in watchdog. --- .../oracle/svm/hosted/DeadlockWatchdog.java | 48 ++++++++++--------- 1 file changed, 26 insertions(+), 22 deletions(-) diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/DeadlockWatchdog.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/DeadlockWatchdog.java index 040eaa642c99..f098d86b8c0e 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/DeadlockWatchdog.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/DeadlockWatchdog.java @@ -82,28 +82,8 @@ void watchdogThread() { while (!stopped) { long now = System.currentTimeMillis(); if (enabled && now >= nextDeadline) { - System.err.println(); - System.err.println("=== Image generator watchdog detected no activity. This can be a sign of a deadlock during image building. Dumping all stack traces. Current time: " + new Date()); - threadDump(); - Runtime runtime = Runtime.getRuntime(); - final long heapSizeUnit = 1024 * 1024; - long usedHeapSize = runtime.totalMemory() / heapSizeUnit; - long freeHeapSize = runtime.freeMemory() / heapSizeUnit; - long maximumHeapSize = runtime.maxMemory() / heapSizeUnit; - System.err.printf("=== Memory statistics (in MB):%n=== Used heap size: %d%n=== Free heap size: %d%n=== Maximum heap size: %d%n", usedHeapSize, freeHeapSize, maximumHeapSize); - System.err.flush(); - - if (watchdogExitOnTimeout) { - System.err.println("=== Image generator watchdog is aborting image generation. To configure the watchdog, use the options " + - SubstrateOptionsParser.commandArgument(SubstrateOptions.DeadlockWatchdogInterval, Integer.toString(watchdogInterval), null) + " and " + - SubstrateOptionsParser.commandArgument(SubstrateOptions.DeadlockWatchdogExitOnTimeout, "+", null)); - /* - * Since there is a likely deadlock somewhere, there is no less intrusive way to - * abort other than a hard exit of the image builder VM. - */ - System.exit(ExitStatus.WATCHDOG_EXIT.getValue()); - - } else { + reportFailureState(); + if (!watchdogExitOnTimeout) { recordActivity(); } } @@ -116,6 +96,30 @@ void watchdogThread() { } } + public void reportFailureState() { + System.err.println(); + System.err.println("=== Image generator watchdog detected no activity. This can be a sign of a deadlock during image building. Dumping all stack traces. Current time: " + new Date()); + threadDump(); + Runtime runtime = Runtime.getRuntime(); + final long heapSizeUnit = 1024 * 1024; + long usedHeapSize = runtime.totalMemory() / heapSizeUnit; + long freeHeapSize = runtime.freeMemory() / heapSizeUnit; + long maximumHeapSize = runtime.maxMemory() / heapSizeUnit; + System.err.printf("=== Memory statistics (in MB):%n=== Used heap size: %d%n=== Free heap size: %d%n=== Maximum heap size: %d%n", usedHeapSize, freeHeapSize, maximumHeapSize); + System.err.flush(); + + if (watchdogExitOnTimeout) { + System.err.println("=== Image generator watchdog is aborting image generation. To configure the watchdog, use the options " + + SubstrateOptionsParser.commandArgument(SubstrateOptions.DeadlockWatchdogInterval, Integer.toString(watchdogInterval), null) + " and " + + SubstrateOptionsParser.commandArgument(SubstrateOptions.DeadlockWatchdogExitOnTimeout, "+", null)); + /* + * Since there is a likely deadlock somewhere, there is no less intrusive way to abort + * other than a hard exit of the image builder VM. + */ + System.exit(ExitStatus.WATCHDOG_EXIT.getValue()); + } + } + public void setEnabled(boolean enable) { if (enable == this.enabled) { return; From cff94e6fe6cc82e52e7984765061187fde683564 Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Thu, 19 Oct 2023 12:57:37 +0200 Subject: [PATCH 5/9] Let CompletionExecutor call `closeDumpHandlers()`. --- .../pointsto/standalone/PointsToAnalyzer.java | 3 +- .../StandalonePointsToAnalysis.java | 12 ++--- .../pointsto/AbstractAnalysisEngine.java | 4 +- .../graal/pointsto/PointsToAnalysis.java | 4 +- .../pointsto/heap/HeapSnapshotVerifier.java | 12 ++--- .../pointsto/util/CompletionExecutor.java | 52 +++++++++++-------- .../ReachabilityAnalysisEngine.java | 16 +++--- .../graal/llvm/LLVMNativeImageCodeCache.java | 2 +- .../core/graal/llvm/LLVMToolchainUtils.java | 4 +- .../graal/llvm/objectfile/LLVMObjectFile.java | 2 +- .../svm/hosted/HostedConfiguration.java | 4 +- .../svm/hosted/NativeImageGenerator.java | 18 +++---- .../hosted/NativeImageGeneratorRunner.java | 9 +--- .../analysis/NativeImagePointsToAnalysis.java | 13 ++--- ...NativeImageReachabilityAnalysisEngine.java | 7 +-- .../oracle/svm/hosted/code/CompileQueue.java | 2 +- .../hosted/image/NativeImageCodeCache.java | 12 ++--- 17 files changed, 90 insertions(+), 86 deletions(-) diff --git a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java index 8785e202b5cf..2bbbca0263a0 100644 --- a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java +++ b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/PointsToAnalyzer.java @@ -151,7 +151,8 @@ private PointsToAnalyzer(String mainEntryClass, OptionValues options) { originalProviders.getPlatformConfigurationProvider(), aMetaAccessExtensionProvider, originalProviders.getLoopsDataProvider()); standaloneHost.initializeProviders(aProviders); analysisName = getAnalysisName(mainEntryClass); - bigbang = new StandalonePointsToAnalysis(options, aUniverse, standaloneHost, aMetaAccess, snippetReflection, aConstantReflection, aProviders.getWordTypes(), new TimerCollection()); + bigbang = new StandalonePointsToAnalysis(options, aUniverse, standaloneHost, aMetaAccess, snippetReflection, aConstantReflection, aProviders.getWordTypes(), debugContext, + new TimerCollection()); standaloneHost.setImageName(analysisName); aUniverse.setBigBang(bigbang); ImageHeap heap = new ImageHeap(); diff --git a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java index 8e769b0bde7d..e48d66606b73 100644 --- a/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java +++ b/substratevm/src/com.oracle.graal.pointsto.standalone/src/com/oracle/graal/pointsto/standalone/StandalonePointsToAnalysis.java @@ -29,10 +29,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; -import jdk.graal.compiler.options.OptionValues; -import jdk.graal.compiler.word.WordTypes; - import com.oracle.graal.pointsto.PointsToAnalysis; import com.oracle.graal.pointsto.api.HostVM; import com.oracle.graal.pointsto.constraints.UnsupportedFeatures; @@ -42,14 +38,18 @@ import com.oracle.graal.pointsto.meta.AnalysisUniverse; import com.oracle.graal.pointsto.util.TimerCollection; +import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.debug.DebugContext; +import jdk.graal.compiler.options.OptionValues; +import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.meta.ConstantReflectionProvider; public class StandalonePointsToAnalysis extends PointsToAnalysis { private final Set addedClinits = ConcurrentHashMap.newKeySet(); public StandalonePointsToAnalysis(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, TimerCollection timerCollection) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new UnsupportedFeatures(), timerCollection, true); + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, DebugContext debugContext, TimerCollection timerCollection) { + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new UnsupportedFeatures(), debugContext, timerCollection, true); } @Override diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java index 687995ec9550..be32d2919e5a 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/AbstractAnalysisEngine.java @@ -100,7 +100,7 @@ public abstract class AbstractAnalysisEngine implements BigBang { @SuppressWarnings("this-escape") public AbstractAnalysisEngine(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, DebugContext debugContext, TimerCollection timerCollection) { this.options = options; this.universe = universe; @@ -109,7 +109,7 @@ public AbstractAnalysisEngine(OptionValues options, AnalysisUniverse universe, H this.metaAccess = metaAccess; this.analysisPolicy = universe.analysisPolicy(); this.hostVM = hostVM; - this.executor = new CompletionExecutor(this); + this.executor = new CompletionExecutor(debugContext, this); this.unsupportedFeatures = unsupportedFeatures; this.processFeaturesTimer = timerCollection.get(TimerCollection.Registry.FEATURES); diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java index 5c6af4e84765..23d0ec04653f 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java @@ -109,9 +109,9 @@ public abstract class PointsToAnalysis extends AbstractAnalysisEngine { @SuppressWarnings("this-escape") public PointsToAnalysis(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, DebugContext debugContext, TimerCollection timerCollection, boolean strengthenGraalGraphs) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection); + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, debugContext, timerCollection); this.typeFlowTimer = timerCollection.createTimer("(typeflow)"); this.strengthenGraalGraphs = strengthenGraalGraphs; diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java index e79f60ca0a6b..d72d31940e9c 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/heap/HeapSnapshotVerifier.java @@ -29,10 +29,6 @@ import java.util.Objects; import java.util.function.Consumer; -import jdk.graal.compiler.options.Option; -import jdk.graal.compiler.options.OptionKey; -import jdk.graal.compiler.options.OptionType; - import com.oracle.graal.pointsto.BigBang; import com.oracle.graal.pointsto.ObjectScanner; import com.oracle.graal.pointsto.ObjectScanner.ReusableSet; @@ -45,6 +41,10 @@ import com.oracle.graal.pointsto.util.CompletionExecutor; import com.oracle.svm.util.LogUtils; +import jdk.graal.compiler.debug.DebugContext; +import jdk.graal.compiler.options.Option; +import jdk.graal.compiler.options.OptionKey; +import jdk.graal.compiler.options.OptionType; import jdk.vm.ci.meta.JavaConstant; public class HeapSnapshotVerifier { @@ -73,8 +73,8 @@ public HeapSnapshotVerifier(BigBang bb, ImageHeap imageHeap, ImageHeapScanner sc verbosity = Options.HeapVerifierVerbosity.getValue(bb.getOptions()); } - public boolean checkHeapSnapshot(UniverseMetaAccess metaAccess, String stage) { - CompletionExecutor executor = new CompletionExecutor(bb); + public boolean checkHeapSnapshot(DebugContext debug, UniverseMetaAccess metaAccess, String stage) { + CompletionExecutor executor = new CompletionExecutor(debug, bb); executor.init(); return checkHeapSnapshot(metaAccess, executor, stage, false); } diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java index 6435a701c420..7e87277bc7a7 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java @@ -65,6 +65,7 @@ private enum State { private final ForkJoinPool executorService; + private final DebugContext debug; private final BigBang bb; private Timing timing; private Object vmConfig; @@ -81,7 +82,8 @@ public interface Timing { void print(); } - public CompletionExecutor(BigBang bb) { + public CompletionExecutor(DebugContext debugContext, BigBang bb) { + this.debug = debugContext.areScopesEnabled() || debugContext.areMetricsEnabled() ? debugContext : null; this.bb = bb; executorService = ForkJoinPool.commonPool(); state = new AtomicReference<>(State.UNUSED); @@ -205,31 +207,37 @@ public long complete() throws InterruptedException { lastPrint = System.nanoTime(); } - while (true) { - assert state.get() == State.STARTED : state.get(); - - boolean quiescent = executorService.awaitQuiescence(100, TimeUnit.MILLISECONDS); - if (timing != null && !quiescent) { - long curTime = System.nanoTime(); - if (curTime - lastPrint > timing.getPrintIntervalNanos()) { - timing.print(); - lastPrint = curTime; + try { + while (true) { + assert state.get() == State.STARTED : state.get(); + + boolean quiescent = executorService.awaitQuiescence(100, TimeUnit.MILLISECONDS); + if (timing != null && !quiescent) { + long curTime = System.nanoTime(); + if (curTime - lastPrint > timing.getPrintIntervalNanos()) { + timing.print(); + lastPrint = curTime; + } } - } - long completed = completedOperations.sum(); - long posted = postedOperations.sum(); - assert completed <= posted : completed + ", " + posted; - if (completed == posted && exceptions.isEmpty()) { - if (timing != null) { - timing.print(); - } + long completed = completedOperations.sum(); + long posted = postedOperations.sum(); + assert completed <= posted : completed + ", " + posted; + if (completed == posted && exceptions.isEmpty()) { + if (timing != null) { + timing.print(); + } - return posted; + return posted; + } + if (!exceptions.isEmpty()) { + setState(State.UNUSED); + throw new ParallelExecutionException(exceptions); + } } - if (!exceptions.isEmpty()) { - setState(State.UNUSED); - throw new ParallelExecutionException(exceptions); + } finally { + if (debug != null) { + debug.closeDumpHandlers(true); } } } diff --git a/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java b/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java index f1eab31b64cc..2e0e854b4834 100644 --- a/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java +++ b/substratevm/src/com.oracle.graal.reachability/src/com/oracle/graal/reachability/ReachabilityAnalysisEngine.java @@ -31,12 +31,6 @@ import java.util.HashSet; import java.util.Set; -import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; -import jdk.graal.compiler.core.common.type.TypedConstant; -import jdk.graal.compiler.nodes.StructuredGraph; -import jdk.graal.compiler.options.OptionValues; -import jdk.graal.compiler.word.WordTypes; - import com.oracle.graal.pointsto.AbstractAnalysisEngine; import com.oracle.graal.pointsto.api.HostVM; import com.oracle.graal.pointsto.constraints.UnsupportedFeatures; @@ -52,6 +46,12 @@ import com.oracle.graal.pointsto.util.TimerCollection; import com.oracle.svm.common.meta.MultiMethod; +import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.core.common.type.TypedConstant; +import jdk.graal.compiler.debug.DebugContext; +import jdk.graal.compiler.nodes.StructuredGraph; +import jdk.graal.compiler.options.OptionValues; +import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.code.BytecodePosition; import jdk.vm.ci.meta.ConstantReflectionProvider; import jdk.vm.ci.meta.JavaConstant; @@ -79,9 +79,9 @@ public abstract class ReachabilityAnalysisEngine extends AbstractAnalysisEngine @SuppressWarnings("this-escape") public ReachabilityAnalysisEngine(OptionValues options, AnalysisUniverse universe, HostVM hostVM, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, TimerCollection timerCollection, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, UnsupportedFeatures unsupportedFeatures, DebugContext debugContext, TimerCollection timerCollection, ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { - super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection); + super(options, universe, hostVM, metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, debugContext, timerCollection); this.executor.init(getTiming()); this.reachabilityTimer = timerCollection.createTimer("(reachability)"); diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java index 6677d3c29752..d93999e889dc 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java @@ -119,7 +119,7 @@ public int codeSizeFor(HostedMethod method) { @SuppressWarnings({"unused", "try"}) public void layoutMethods(DebugContext debug, BigBang bb) { try (Indent indent = debug.logAndIndent("layout methods")) { - BatchExecutor executor = new BatchExecutor(bb); + BatchExecutor executor = new BatchExecutor(debug, bb); try (StopTimer t = TimerCollection.createTimerAndStart("(bitcode)")) { writeBitcode(executor); } diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java index d7b746d668fe..b10092fff482 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMToolchainUtils.java @@ -169,8 +169,8 @@ public static void llvmCleanupStackMaps(DebugContext debug, String inputPath, Pa public static final class BatchExecutor { private CompletionExecutor executor; - public BatchExecutor(BigBang bb) { - this.executor = new CompletionExecutor(bb); + public BatchExecutor(DebugContext debug, BigBang bb) { + this.executor = new CompletionExecutor(debug, bb); executor.init(); } diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java index bcce8a587fea..bc3801c339a8 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/objectfile/LLVMObjectFile.java @@ -191,7 +191,7 @@ public final void write(DebugContext context, Path outputFile) throws IOExceptio writeParts(); - BatchExecutor batchExecutor = new BatchExecutor(bb); + BatchExecutor batchExecutor = new BatchExecutor(context, bb); compileBitcodeBatches(batchExecutor, context, numBatches); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java index e7255096bfbf..769a01f4c27a 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/HostedConfiguration.java @@ -172,8 +172,8 @@ public SVMHost createHostVM(OptionValues options, ClassLoader classLoader, Class return new SVMHost(options, classLoader, classInitializationSupport, automaticSubstitutions, platform); } - public CompileQueue createCompileQueue(DebugContext debug, FeatureHandler featureHandler, HostedUniverse hostedUniverse, - RuntimeConfiguration runtimeConfiguration, boolean deoptimizeAll, SnippetReflectionProvider aSnippetReflection) { + public CompileQueue createCompileQueue(DebugContext debug, FeatureHandler featureHandler, HostedUniverse hostedUniverse, RuntimeConfiguration runtimeConfiguration, boolean deoptimizeAll, + SnippetReflectionProvider aSnippetReflection) { return new CompileQueue(debug, featureHandler, hostedUniverse, runtimeConfiguration, deoptimizeAll, aSnippetReflection); } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java index 1714fd92b78b..135503c4fbab 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGenerator.java @@ -574,9 +574,7 @@ public static void clearSystemPropertiesForImage() { } @SuppressWarnings("try") - protected void doRun(Map entryPoints, - JavaMainSupport javaMainSupport, String imageName, NativeImageKind k, - SubstitutionProcessor harnessSubstitutions) { + protected void doRun(Map entryPoints, JavaMainSupport javaMainSupport, String imageName, NativeImageKind k, SubstitutionProcessor harnessSubstitutions) { List hostedEntryPoints = new ArrayList<>(); OptionValues options = HostedOptionValues.singleton(); @@ -676,7 +674,7 @@ protected void doRun(Map entryPoints, ImageSingletons.lookup(TemporaryBuildDirectoryProvider.class).getTemporaryBuildDirectory()); codeCache.layoutConstants(); codeCache.layoutMethods(debug, bb); - codeCache.buildRuntimeMetadata(bb.getSnippetReflectionProvider()); + codeCache.buildRuntimeMetadata(debug, bb.getSnippetReflectionProvider()); } AfterCompilationAccessImpl config = new AfterCompilationAccessImpl(featureHandler, loader, aUniverse, hUniverse, compileQueue.getCompilations(), codeCache, heap, debug, @@ -686,7 +684,7 @@ protected void doRun(Map entryPoints, } /* Re-run shadow heap verification after compilation. */ - aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, "after compilation"); + aUniverse.getHeapVerifier().checkHeapSnapshot(debug, hMetaAccess, "after compilation"); CodeCacheProvider codeCacheProvider = runtimeConfiguration.getBackendForNormalMethod().getProviders().getCodeCache(); reporter.printCreationStart(); @@ -705,7 +703,7 @@ protected void doRun(Map entryPoints, featureHandler.forEachFeature(feature -> feature.afterHeapLayout(config)); /* Re-run shadow heap verification after heap layout. */ - aUniverse.getHeapVerifier().checkHeapSnapshot(hMetaAccess, "after heap layout"); + aUniverse.getHeapVerifier().checkHeapSnapshot(debug, hMetaAccess, "after heap layout"); createAbstractImage(k, hostedEntryPoints, heap, hMetaAccess, codeCache); @@ -925,7 +923,7 @@ protected void setupNativeImage(OptionValues options, Map")); } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java index 6411ed167001..063263b7ef68 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImagePointsToAnalysis.java @@ -28,10 +28,6 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; -import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; -import jdk.graal.compiler.options.OptionValues; -import jdk.graal.compiler.word.WordTypes; - import com.oracle.graal.pointsto.PointsToAnalysis; import com.oracle.graal.pointsto.constraints.UnsupportedFeatures; import com.oracle.graal.pointsto.flow.MethodFlowsGraph; @@ -50,6 +46,10 @@ import com.oracle.svm.hosted.meta.HostedType; import com.oracle.svm.hosted.substitute.AnnotationSubstitutionProcessor; +import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.debug.DebugContext; +import jdk.graal.compiler.options.OptionValues; +import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.code.BytecodePosition; import jdk.vm.ci.meta.ConstantReflectionProvider; import jdk.vm.ci.meta.ResolvedJavaType; @@ -66,8 +66,9 @@ public NativeImagePointsToAnalysis(OptionValues options, AnalysisUniverse univer AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, AnnotationSubstitutionProcessor annotationSubstitutionProcessor, UnsupportedFeatures unsupportedFeatures, - TimerCollection timerCollection) { - super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, timerCollection, SubstrateOptions.parseOnce()); + DebugContext debugContext, TimerCollection timerCollection) { + super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, unsupportedFeatures, debugContext, timerCollection, + SubstrateOptions.parseOnce()); this.annotationSubstitutionProcessor = annotationSubstitutionProcessor; dynamicHubInitializer = new DynamicHubInitializer(this); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java index 0330bd762c8e..7d013ac2f524 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/analysis/NativeImageReachabilityAnalysisEngine.java @@ -36,6 +36,7 @@ import com.oracle.svm.hosted.substitute.AnnotationSubstitutionProcessor; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; +import jdk.graal.compiler.debug.DebugContext; import jdk.graal.compiler.options.OptionValues; import jdk.graal.compiler.word.WordTypes; import jdk.vm.ci.meta.ConstantReflectionProvider; @@ -49,9 +50,9 @@ public class NativeImageReachabilityAnalysisEngine extends ReachabilityAnalysisE @SuppressWarnings("this-escape") public NativeImageReachabilityAnalysisEngine(OptionValues options, AnalysisUniverse universe, AnalysisMetaAccess metaAccess, SnippetReflectionProvider snippetReflectionProvider, - ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, AnnotationSubstitutionProcessor annotationSubstitutionProcessor, TimerCollection timerCollection, - ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { - super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new SubstrateUnsupportedFeatures(), timerCollection, + ConstantReflectionProvider constantReflectionProvider, WordTypes wordTypes, AnnotationSubstitutionProcessor annotationSubstitutionProcessor, DebugContext debugContext, + TimerCollection timerCollection, ReachabilityMethodProcessingHandler reachabilityMethodProcessingHandler) { + super(options, universe, universe.hostVM(), metaAccess, snippetReflectionProvider, constantReflectionProvider, wordTypes, new SubstrateUnsupportedFeatures(), debugContext, timerCollection, reachabilityMethodProcessingHandler); this.annotationSubstitutionProcessor = annotationSubstitutionProcessor; this.strengthenGraalGraphs = SubstrateOptions.parseOnce(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java index 71a3028aca69..52fac102b699 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/code/CompileQueue.java @@ -367,7 +367,7 @@ public CompileQueue(DebugContext debug, FeatureHandler featureHandler, HostedUni this.metaAccess = runtimeConfiguration.getProviders().getMetaAccess(); this.deoptimizeAll = deoptimizeAll; this.dataCache = new ConcurrentHashMap<>(); - this.executor = new CompletionExecutor(universe.getBigBang()); + this.executor = new CompletionExecutor(debug, universe.getBigBang()); this.featureHandler = featureHandler; this.snippetReflection = snippetReflection; this.graphTransplanter = createGraphTransplanter(); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java index ece30a9b5148..768433377bc8 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/NativeImageCodeCache.java @@ -262,11 +262,11 @@ public int getAlignedConstantsSize() { return ConfigurationValues.getObjectLayout().alignUp(getConstantsSize()); } - public void buildRuntimeMetadata(SnippetReflectionProvider snippetReflectionProvider) { - buildRuntimeMetadata(snippetReflectionProvider, new MethodPointer(getFirstCompilation().getLeft(), true), WordFactory.signed(getCodeAreaSize())); + public void buildRuntimeMetadata(DebugContext debug, SnippetReflectionProvider snippetReflectionProvider) { + buildRuntimeMetadata(debug, snippetReflectionProvider, new MethodPointer(getFirstCompilation().getLeft(), true), WordFactory.signed(getCodeAreaSize())); } - protected void buildRuntimeMetadata(SnippetReflectionProvider snippetReflection, CFunctionPointer firstMethod, UnsignedWord codeSize) { + protected void buildRuntimeMetadata(DebugContext debug, SnippetReflectionProvider snippetReflection, CFunctionPointer firstMethod, UnsignedWord codeSize) { // Build run-time metadata. HostedFrameInfoCustomization frameInfoCustomization = new HostedFrameInfoCustomization(); CodeInfoEncoder.Encoders encoders = new CodeInfoEncoder.Encoders(); @@ -424,7 +424,7 @@ protected void buildRuntimeMetadata(SnippetReflectionProvider snippetReflection, verifyDeoptEntries(imageCodeInfo); } - assert verifyMethods(hUniverse, codeInfoEncoder, imageCodeInfo); + assert verifyMethods(debug, hUniverse, codeInfoEncoder, imageCodeInfo); } protected HostedImageCodeInfo installCodeInfo(SnippetReflectionProvider snippetReflection, CFunctionPointer firstMethod, UnsignedWord codeSize, CodeInfoEncoder codeInfoEncoder, @@ -567,12 +567,12 @@ private static boolean error(HostedMethod method, long encodedBci, String msg) { return true; } - protected boolean verifyMethods(HostedUniverse hUniverse, CodeInfoEncoder codeInfoEncoder, CodeInfo codeInfo) { + protected boolean verifyMethods(DebugContext debug, HostedUniverse hUniverse, CodeInfoEncoder codeInfoEncoder, CodeInfo codeInfo) { /* * Run method verification in parallel to reduce computation time. */ BigBang bb = hUniverse.getBigBang(); - CompletionExecutor executor = new CompletionExecutor(bb); + CompletionExecutor executor = new CompletionExecutor(debug, bb); try { executor.init(); executor.start(); From 932b40f90433e86ce07cf22bf74f837cae05b14a Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Thu, 19 Oct 2023 13:08:16 +0200 Subject: [PATCH 6/9] Set parallelism of common pool within builder. --- .../native-image/BuildOutput.md | 2 +- .../graal/llvm/LLVMNativeImageCodeCache.java | 4 +- .../com/oracle/svm/core/SubstrateOptions.java | 11 ---- .../com/oracle/svm/driver/NativeImage.java | 30 ----------- .../hosted/NativeImageGeneratorRunner.java | 6 +++ .../oracle/svm/hosted/NativeImageOptions.java | 54 +++++++++++++++++++ .../oracle/svm/hosted/ProgressReporter.java | 6 +-- 7 files changed, 66 insertions(+), 47 deletions(-) diff --git a/docs/reference-manual/native-image/BuildOutput.md b/docs/reference-manual/native-image/BuildOutput.md index 39c8fc43a2c1..b26b3b5abd3c 100644 --- a/docs/reference-manual/native-image/BuildOutput.md +++ b/docs/reference-manual/native-image/BuildOutput.md @@ -137,7 +137,7 @@ By default, the build process tries to only use free memory (to avoid memory pre If less than 8GB of memory are free, the build process falls back to use 85% of total memory. Therefore, consider freeing up memory if your machine is slow during a build, for example, by closing applications that you do not need. -By default, the build process uses all available CPU cores to maximize speed. +By default, the build process uses all available processors to maximize speed, but not more than 32 threads. Use the `--parallelism` option to set the number of threads explicitly (for example, `--parallelism=4`). Use fewer threads to reduce load on your system as well as memory consumption (at the cost of a slower build process). diff --git a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java index d93999e889dc..1611f7ce4ba4 100644 --- a/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java +++ b/substratevm/src/com.oracle.svm.core.graal.llvm/src/com/oracle/svm/core/graal/llvm/LLVMNativeImageCodeCache.java @@ -45,7 +45,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import com.oracle.svm.core.SubstrateOptions; import org.graalvm.collections.Pair; import jdk.graal.compiler.code.CompilationResult; import jdk.graal.compiler.core.common.NumUtil; @@ -72,6 +71,7 @@ import com.oracle.svm.core.graal.llvm.util.LLVMStackMapInfo; import com.oracle.svm.core.heap.SubstrateReferenceMap; import com.oracle.svm.core.jdk.UninterruptibleUtils.AtomicInteger; +import com.oracle.svm.hosted.NativeImageOptions; import com.oracle.svm.hosted.image.NativeImage.NativeTextSectionImpl; import com.oracle.svm.hosted.image.NativeImageCodeCache; import com.oracle.svm.hosted.image.NativeImageHeap; @@ -153,7 +153,7 @@ private void writeBitcode(BatchExecutor executor) { private int createBitcodeBatches(BatchExecutor executor, DebugContext debug) { batchSize = LLVMOptions.LLVMMaxFunctionsPerBatch.getValue(); - int numThreads = SubstrateOptions.NumberOfThreads.getValue(); + int numThreads = NativeImageOptions.getActualNumberOfThreads(); int idealSize = NumUtil.divideAndRoundUp(methodIndex.length, numThreads); if (idealSize < batchSize) { batchSize = idealSize; diff --git a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java index 03788b94e10a..2948aacaa928 100644 --- a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java +++ b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/SubstrateOptions.java @@ -58,7 +58,6 @@ import com.oracle.svm.core.option.SubstrateOptionsParser; import com.oracle.svm.core.thread.VMOperationControl; import com.oracle.svm.core.util.UserError; -import com.oracle.svm.core.util.VMError; import com.oracle.svm.util.LogUtils; import com.oracle.svm.util.ModuleSupport; import com.oracle.svm.util.ReflectionUtil; @@ -107,16 +106,6 @@ public static boolean parseOnce() { @Option(help = "Name of the output file to be generated", type = OptionType.User)// public static final HostedOptionKey Name = new HostedOptionKey<>(""); - /** - * Configures the number of threads of the common pool (see driver). - */ - @APIOption(name = "parallelism")// - @Option(help = "The maximum number of threads to use concurrently during native image generation.")// - public static final HostedOptionKey NumberOfThreads = new HostedOptionKey<>(Math.max(1, Math.min(Runtime.getRuntime().availableProcessors(), 32)), key -> { - int numberOfThreads = key.getValue(); - VMError.guarantee(numberOfThreads >= 1, "Number of threads must be at least 1. Validation should have happened in driver."); - }); - @APIOption(name = "shared")// @Option(help = "Build shared library")// public static final HostedOptionKey SharedLibrary = new HostedOptionKey<>(false); diff --git a/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java b/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java index 5641e7ecf1a6..3057cce3bb9b 100644 --- a/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java +++ b/substratevm/src/com.oracle.svm.driver/src/com/oracle/svm/driver/NativeImage.java @@ -268,8 +268,6 @@ private static String oR(OptionKey option) { final String oHInspectServerContentPath = oH(PointstoOptions.InspectServerContentPath); final String oHDeadlockWatchdogInterval = oH(SubstrateOptions.DeadlockWatchdogInterval); - static final String oHNumberOfThreads = oH(SubstrateOptions.NumberOfThreads); - final Map imageBuilderEnvironment = new HashMap<>(); private final ArrayList imageBuilderArgs = new ArrayList<>(); private final LinkedHashSet imageBuilderModulePath = new LinkedHashSet<>(); @@ -1095,20 +1093,6 @@ private int completeImageBuild() { } imageClasspath.addAll(customImageClasspath); - Integer maxNumberOfThreads = getMaxNumberOfThreads(); - if (maxNumberOfThreads != null) { - if (maxNumberOfThreads >= 1) { - /* - * maxNumberOfThreads - 1 because the main thread always helps to process tasks. In - * single-threaded mode (parallelism=0 for common pool), only the main thread - * processes tasks. - */ - imageBuilderJavaArgs.add("-Djava.util.concurrent.ForkJoinPool.common.parallelism=" + (maxNumberOfThreads - 1)); - } else { - throw showError("The number of threads was set to " + maxNumberOfThreads + ". Please set the '--parallelism' option to at least 1."); - } - } - /* * Work around "JDK-8315810: Reimplement * sun.reflect.ReflectionFactory::newConstructorForSerialization with method handles" @@ -1359,20 +1343,6 @@ private static Boolean getHostedOptionFinalBooleanArgumentValue(List arg return result; } - private Integer getMaxNumberOfThreads() { - String numberOfThreadsValue = getHostedOptionFinalArgumentValue(imageBuilderArgs, oHNumberOfThreads); - if (numberOfThreadsValue != null) { - try { - return Integer.parseInt(numberOfThreadsValue); - } catch (NumberFormatException e) { - /* Validated already by CommonOptionParser. */ - throw VMError.shouldNotReachHere(e); - } - } else { - return null; - } - } - private boolean shouldAddCWDToCP() { if (config.buildFallbackImage() || printFlagsOptionQuery != null || printFlagsWithExtraHelpOptionQuery != null) { return false; diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGeneratorRunner.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGeneratorRunner.java index 2d561753051e..7de0c7713c42 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGeneratorRunner.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageGeneratorRunner.java @@ -292,6 +292,12 @@ public static ImageClassLoader installNativeImageClassLoader(String[] classpath, */ NativeImageGenerator.setSystemPropertiesForImageEarly(); + /* + * Size the common pool before creating the image class loader because it is the first + * component to use the common pool. + */ + NativeImageOptions.setCommonPoolParallelism(nativeImageClassLoaderSupport.getParsedHostedOptions()); + return new ImageClassLoader(NativeImageGenerator.getTargetPlatform(nativeImageClassLoader), nativeImageClassLoaderSupport); } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageOptions.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageOptions.java index ac2626469a98..56cd319598a5 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageOptions.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/NativeImageOptions.java @@ -31,7 +31,9 @@ import java.nio.file.Paths; import java.text.SimpleDateFormat; import java.util.Date; +import java.util.concurrent.ForkJoinPool; +import com.oracle.svm.util.LogUtils; import org.graalvm.collections.EconomicMap; import jdk.graal.compiler.options.Option; import jdk.graal.compiler.options.OptionKey; @@ -186,6 +188,58 @@ public static CStandards getCStandard() { } } + /** + * Configures the number of threads of the common pool (see driver). + */ + private static final String PARALLELISM_OPTION_NAME = "parallelism"; + @APIOption(name = PARALLELISM_OPTION_NAME)// + @Option(help = "The maximum number of threads to use concurrently during native image generation.")// + public static final HostedOptionKey NumberOfThreads = new HostedOptionKey<>(Math.max(1, Math.min(Runtime.getRuntime().availableProcessors(), 32)), key -> { + int numberOfThreads = key.getValue(); + if (numberOfThreads < 1) { + throw UserError.abort("The number of threads was set to %s. Please set the '--%s' option to at least 1.", numberOfThreads, PARALLELISM_OPTION_NAME); + } + }); + + public static int getActualNumberOfThreads() { + int commonThreadParallelism = ForkJoinPool.getCommonPoolParallelism(); + if (NumberOfThreads.getValue() == 1) { + assert commonThreadParallelism == 1 : "Disabled common pool expected to report parallelism of 1"; + commonThreadParallelism = 0; /* A disabled common pool has no actual threads */ + } + /* + * Main thread plus common pool threads. setCommonPoolParallelism() asserts that this number + * matches NumberOfThreads. + */ + return 1 + commonThreadParallelism; + } + + public static void setCommonPoolParallelism(OptionValues optionValues) { + if (NativeImageOptions.NumberOfThreads.hasBeenSet(optionValues)) { + /* + * The main thread always helps to process tasks submitted to the common pool (e.g., see + * ForkJoinPool#awaitTermination()), so subtract one from the number of threads. The + * common pool can be disabled "by setting the parallelism property to zero" (see + * ForkJoinPool's javadoc). + */ + int numberOfCommonPoolThreads = NativeImageOptions.NumberOfThreads.getValue(optionValues) - 1; + String commonPoolParallelismProperty = "java.util.concurrent.ForkJoinPool.common.parallelism"; + assert System.getProperty(commonPoolParallelismProperty) == null : commonPoolParallelismProperty + " already set"; + System.setProperty(commonPoolParallelismProperty, "" + numberOfCommonPoolThreads); + int actualCommonPoolParallelism = ForkJoinPool.commonPool().getParallelism(); + /* + * getParallelism() returns at least 1, even in single-threaded mode where common pool + * is disabled. + */ + boolean isSingleThreadedMode = numberOfCommonPoolThreads == 0 && actualCommonPoolParallelism == 1; + if (!isSingleThreadedMode && actualCommonPoolParallelism != numberOfCommonPoolThreads) { + String warning = "Failed to set parallelism of common pool (actual parallelism is %s).".formatted(actualCommonPoolParallelism); + assert false : warning; + LogUtils.warning(warning); + } + } + } + @Option(help = "Deprecated, option no longer has any effect", deprecated = true, deprecationMessage = "Please use '--parallelism' instead.")// public static final HostedOptionKey NumberOfAnalysisThreads = new HostedOptionKey<>(-1); diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java index 309031cd0a71..d36baa66971a 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/ProgressReporter.java @@ -392,13 +392,13 @@ private void printResourceInfo() { maxHeapSuffix = "set via '%s'".formatted(xmxValueOrNull); } - int maxNumberOfThreads = SubstrateOptions.NumberOfThreads.getValue(); + int maxNumberOfThreads = NativeImageOptions.getActualNumberOfThreads(); recordJsonMetric(ResourceUsageKey.PARALLELISM, maxNumberOfThreads); int availableProcessors = runtime.availableProcessors(); recordJsonMetric(ResourceUsageKey.CPU_CORES_TOTAL, availableProcessors); String maxNumberOfThreadsSuffix = "determined at start"; - if (SubstrateOptions.NumberOfThreads.hasBeenSet()) { - maxNumberOfThreadsSuffix = "set via '%s'".formatted(SubstrateOptionsParser.commandArgument(SubstrateOptions.NumberOfThreads, Integer.toString(maxNumberOfThreads))); + if (NativeImageOptions.NumberOfThreads.hasBeenSet()) { + maxNumberOfThreadsSuffix = "set via '%s'".formatted(SubstrateOptionsParser.commandArgument(NativeImageOptions.NumberOfThreads, Integer.toString(maxNumberOfThreads))); } l().printLineSeparator(); From af128f66959d50c80005c61b6550b9d066223018 Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Thu, 19 Oct 2023 13:11:11 +0200 Subject: [PATCH 7/9] Drop ununsed code. --- .../graal/pointsto/PointsToAnalysis.java | 20 ------ .../DisallowedImageHeapObjectFeature.java | 5 -- .../svm/util/ImageGeneratorThreadMarker.java | 67 ------------------- 3 files changed, 92 deletions(-) delete mode 100644 substratevm/src/com.oracle.svm.util/src/com/oracle/svm/util/ImageGeneratorThreadMarker.java diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java index 23d0ec04653f..cda851aa7c95 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/PointsToAnalysis.java @@ -37,8 +37,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.ForkJoinWorkerThread; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongArray; import java.util.function.Consumer; @@ -75,7 +73,6 @@ import com.oracle.graal.pointsto.util.TimerCollection; import com.oracle.svm.common.meta.MultiMethod; import com.oracle.svm.util.ClassUtil; -import com.oracle.svm.util.ImageGeneratorThreadMarker; import jdk.graal.compiler.api.replacements.SnippetReflectionProvider; import jdk.graal.compiler.debug.DebugContext; @@ -723,21 +720,4 @@ public void print() { System.out.println(); } } - - private static class SubstrateWorkerThread extends ForkJoinWorkerThread - implements ImageGeneratorThreadMarker { - private final DebugContext debug; - - SubstrateWorkerThread(ForkJoinPool pool, DebugContext debug) { - super(pool); - this.debug = debug; - } - - @Override - protected void onTermination(Throwable exception) { - if (debug != null) { - debug.closeDumpHandlers(true); - } - } - } } diff --git a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/DisallowedImageHeapObjectFeature.java b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/DisallowedImageHeapObjectFeature.java index 41597934921c..959bd979b12c 100644 --- a/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/DisallowedImageHeapObjectFeature.java +++ b/substratevm/src/com.oracle.svm.hosted/src/com/oracle/svm/hosted/image/DisallowedImageHeapObjectFeature.java @@ -50,7 +50,6 @@ import com.oracle.svm.hosted.FeatureImpl; import com.oracle.svm.hosted.classinitialization.ClassInitializationOptions; import com.oracle.svm.hosted.classinitialization.ClassInitializationSupport; -import com.oracle.svm.util.ImageGeneratorThreadMarker; /** * Complain if there are types that can not move from the image generator heap to the image heap. @@ -113,10 +112,6 @@ private static String[] getDisallowedSubstrings(String... substrings) { } private Object replacer(Object original) { - if (original instanceof Thread && original instanceof ImageGeneratorThreadMarker) { - return ((ImageGeneratorThreadMarker) original).asTerminated(); - } - checkDisallowedMBeanObjects(original); if (original instanceof String && disallowedSubstrings != null) { diff --git a/substratevm/src/com.oracle.svm.util/src/com/oracle/svm/util/ImageGeneratorThreadMarker.java b/substratevm/src/com.oracle.svm.util/src/com/oracle/svm/util/ImageGeneratorThreadMarker.java deleted file mode 100644 index d11e1e338fd1..000000000000 --- a/substratevm/src/com.oracle.svm.util/src/com/oracle/svm/util/ImageGeneratorThreadMarker.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright (c) 2019, 2019, 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 com.oracle.svm.util; - -import org.graalvm.nativeimage.Platform; -import org.graalvm.nativeimage.Platforms; - -/** - * Marker interface to identify threads that are only used by SubstateVM infrastructure and will not - * be present in the image heap at run time. Each of such threads has a {@link #asTerminated() way} - * to obtain the terminated replacement of itself. - *

- * Not intented to be implemented by customer threads. Allowing background threads that are not - * terminated can lead to very hard to debug errors. If you have background threads running while - * the static analysis is running, then bad race conditions can happen: the static analysis looks at - * a heap snapshot at a "random" time from the point of view of the background thread. If the - * background thread makes new objects available after the last heap snapshot, then the static - * analysis misses objects of the image heap, which leads to either strange errors during image - * generation (when the image heap is written), or even worse strange behavior at run time (when a - * field is, e.g., constant folded to {@code null} because the static analysis only saw the - * {@code null} value). - */ -@Platforms(Platform.HOSTED_ONLY.class) -public interface ImageGeneratorThreadMarker /* extends Thread */ { - default Thread asTerminated() { - return TerminatedThread.SINGLETON; - } -} - -final class TerminatedThread extends Thread { - static final TerminatedThread SINGLETON; - static { - SINGLETON = new TerminatedThread("Terminated Infrastructure Thread"); - SINGLETON.start(); - try { - SINGLETON.join(); - } catch (InterruptedException ex) { - throw new IllegalStateException(ex); - } - } - - TerminatedThread(String name) { - super(name); - } -} From 589c73ac58cec2f794ea19532224be5be6c8911b Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Fri, 20 Oct 2023 08:57:06 +0200 Subject: [PATCH 8/9] Drop executorService field from CompletionExecutor. --- .../graal/pointsto/util/CompletionExecutor.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java index 7e87277bc7a7..a57aa9dda6c1 100644 --- a/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java +++ b/substratevm/src/com.oracle.graal.pointsto/src/com/oracle/graal/pointsto/util/CompletionExecutor.java @@ -34,15 +34,14 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; +import com.oracle.graal.pointsto.BigBang; + import jdk.graal.compiler.debug.DebugContext; import jdk.graal.compiler.debug.DebugContext.Activation; import jdk.graal.compiler.debug.DebugContext.Description; import jdk.graal.compiler.debug.DebugContext.Scope; import jdk.graal.compiler.debug.DebugHandlersFactory; import jdk.graal.compiler.options.OptionValues; - -import com.oracle.graal.pointsto.BigBang; - import jdk.vm.ci.common.JVMCIError; /** @@ -63,8 +62,6 @@ private enum State { private List postedBeforeStart; private final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); - private final ForkJoinPool executorService; - private final DebugContext debug; private final BigBang bb; private Timing timing; @@ -85,7 +82,6 @@ public interface Timing { public CompletionExecutor(DebugContext debugContext, BigBang bb) { this.debug = debugContext.areScopesEnabled() || debugContext.areMetricsEnabled() ? debugContext : null; this.bb = bb; - executorService = ForkJoinPool.commonPool(); state = new AtomicReference<>(State.UNUSED); postedOperations = new LongAdder(); completedOperations = new LongAdder(); @@ -155,7 +151,7 @@ public void execute(DebugContextRunnable command) { } private void executeService(DebugContextRunnable command) { - executorService.execute(() -> executeCommand(command)); + ForkJoinPool.commonPool().execute(() -> executeCommand(command)); } @SuppressWarnings("try") @@ -211,7 +207,7 @@ public long complete() throws InterruptedException { while (true) { assert state.get() == State.STARTED : state.get(); - boolean quiescent = executorService.awaitQuiescence(100, TimeUnit.MILLISECONDS); + boolean quiescent = ForkJoinPool.commonPool().awaitQuiescence(100, TimeUnit.MILLISECONDS); if (timing != null && !quiescent) { long curTime = System.nanoTime(); if (curTime - lastPrint > timing.getPrintIntervalNanos()) { @@ -247,7 +243,7 @@ public long getPostedOperations() { } public void shutdown() { - assert !executorService.hasQueuedSubmissions() : "There should be no queued submissions on shutdown."; + assert !ForkJoinPool.commonPool().hasQueuedSubmissions() : "There should be no queued submissions on shutdown."; assert completedOperations.sum() == postedOperations.sum() : "Posted operations (" + postedOperations.sum() + ") must match completed (" + completedOperations.sum() + ") operations"; setState(State.UNUSED); } From 72876c81526869f201ac6a1ddedf7ffdd5c5c5fa Mon Sep 17 00:00:00 2001 From: Fabio Niephaus Date: Fri, 20 Oct 2023 11:21:16 +0200 Subject: [PATCH 9/9] Ensure option descriptor is loaded when accessed. --- .../jdk/graal/compiler/options/OptionKey.java | 16 +++++++++++++++- .../svm/core/option/SubstrateOptionsParser.java | 6 ++++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/compiler/src/jdk.graal.compiler/src/jdk/graal/compiler/options/OptionKey.java b/compiler/src/jdk.graal.compiler/src/jdk/graal/compiler/options/OptionKey.java index 12471587bdef..8e485bb53ec1 100644 --- a/compiler/src/jdk.graal.compiler/src/jdk/graal/compiler/options/OptionKey.java +++ b/compiler/src/jdk.graal.compiler/src/jdk/graal/compiler/options/OptionKey.java @@ -50,12 +50,26 @@ public final void setDescriptor(OptionDescriptor descriptor) { /** * Returns the descriptor for this option, if it has been set by - * {@link #setDescriptor(OptionDescriptor)}. + * {@link #setDescriptor(OptionDescriptor)}. As descriptors are loaded lazily, this method will + * return {@code null} if the descriptors have not been loaded. Use {@link #loadDescriptor} + * instead to ensure a non-null descriptor is returned if available. */ public final OptionDescriptor getDescriptor() { return descriptor; } + /** + * Returns the descriptor for this option, triggering loading of descriptors if this descriptor + * is null. Note that it's still possible for this method to return null if this option does not + * have a descriptor created by a service loader. + */ + public final OptionDescriptor loadDescriptor() { + if (descriptor == null) { + Lazy.init(); + } + return descriptor; + } + /** * Checks that a descriptor exists for this key after triggering loading of descriptors. */ diff --git a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/option/SubstrateOptionsParser.java b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/option/SubstrateOptionsParser.java index 9b4bb45e80f0..5c2151bddc45 100644 --- a/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/option/SubstrateOptionsParser.java +++ b/substratevm/src/com.oracle.svm.core/src/com/oracle/svm/core/option/SubstrateOptionsParser.java @@ -175,16 +175,18 @@ public static String commandArgument(OptionKey option, String value) { */ @Platforms(Platform.HOSTED_ONLY.class) public static String commandArgument(OptionKey option, String value, String apiOptionName) { + /* Ensure descriptor is loaded */ + OptionDescriptor optionDescriptor = option.loadDescriptor(); Field field; try { - field = option.getDescriptor().getDeclaringClass().getDeclaredField(option.getDescriptor().getFieldName()); + field = optionDescriptor.getDeclaringClass().getDeclaredField(optionDescriptor.getFieldName()); } catch (ReflectiveOperationException ex) { throw VMError.shouldNotReachHere(ex); } APIOption[] apiOptions = field.getAnnotationsByType(APIOption.class); - if (option.getDescriptor().getOptionValueType() == Boolean.class) { + if (optionDescriptor.getOptionValueType() == Boolean.class) { VMError.guarantee(value.equals("+") || value.equals("-"), "Boolean option value can be only + or -"); for (APIOption apiOption : apiOptions) { String selected = selectVariant(apiOption, apiOptionName);