changeset 10672:e7d07c9bb779

Removed priority compilation queue. Fixed another race condition in the compilation queue. Removed Graal-specific compilation policy.
author Christian Haeubl <haeubl@ssw.jku.at>
date Tue, 09 Jul 2013 17:53:58 +0200
parents dcee58529a1c
children d59e7f94f28d
files graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilationTask.java graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompileTheWorld.java graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilerThread.java graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompiler.java graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompilerImpl.java graal/com.oracle.graal.phases/src/com/oracle/graal/phases/GraalOptions.java src/cpu/x86/vm/templateInterpreter_x86_64.cpp src/share/vm/classfile/vmSymbols.hpp src/share/vm/graal/graalCompiler.cpp src/share/vm/graal/graalVMToCompiler.cpp src/share/vm/graal/graalVMToCompiler.hpp src/share/vm/oops/methodCounters.cpp src/share/vm/oops/methodCounters.hpp src/share/vm/runtime/compilationPolicy.cpp
diffstat 14 files changed, 46 insertions(+), 257 deletions(-) [+]
line wrap: on
line diff
--- a/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilationTask.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilationTask.java	Tue Jul 09 17:53:58 2013 +0200
@@ -36,7 +36,6 @@
 import com.oracle.graal.compiler.*;
 import com.oracle.graal.debug.*;
 import com.oracle.graal.debug.internal.*;
-import com.oracle.graal.hotspot.bridge.*;
 import com.oracle.graal.hotspot.meta.*;
 import com.oracle.graal.nodes.*;
 import com.oracle.graal.nodes.spi.*;
@@ -44,7 +43,7 @@
 import com.oracle.graal.phases.common.*;
 import com.oracle.graal.phases.tiers.*;
 
-public final class CompilationTask implements Runnable, Comparable<CompilationTask> {
+public final class CompilationTask implements Runnable {
 
     public static final ThreadLocal<Boolean> withinEnqueue = new ThreadLocal<Boolean>() {
 
@@ -55,7 +54,7 @@
     };
 
     private enum CompilationStatus {
-        Queued, Running, Canceled
+        Queued, Running
     }
 
     private final HotSpotGraalRuntime graalRuntime;
@@ -65,16 +64,15 @@
     private final HotSpotResolvedJavaMethod method;
     private final int entryBCI;
     private final int id;
-    private final int priority;
     private final AtomicReference<CompilationStatus> status;
 
     private StructuredGraph graph;
 
-    public static CompilationTask create(HotSpotGraalRuntime graalRuntime, PhasePlan plan, OptimisticOptimizations optimisticOpts, HotSpotResolvedJavaMethod method, int entryBCI, int id, int priority) {
-        return new CompilationTask(graalRuntime, plan, optimisticOpts, method, entryBCI, id, priority);
+    public static CompilationTask create(HotSpotGraalRuntime graalRuntime, PhasePlan plan, OptimisticOptimizations optimisticOpts, HotSpotResolvedJavaMethod method, int entryBCI, int id) {
+        return new CompilationTask(graalRuntime, plan, optimisticOpts, method, entryBCI, id);
     }
 
-    private CompilationTask(HotSpotGraalRuntime graalRuntime, PhasePlan plan, OptimisticOptimizations optimisticOpts, HotSpotResolvedJavaMethod method, int entryBCI, int id, int priority) {
+    private CompilationTask(HotSpotGraalRuntime graalRuntime, PhasePlan plan, OptimisticOptimizations optimisticOpts, HotSpotResolvedJavaMethod method, int entryBCI, int id) {
         assert id >= 0;
         this.graalRuntime = graalRuntime;
         this.plan = plan;
@@ -83,7 +81,6 @@
         this.optimisticOpts = optimisticOpts;
         this.entryBCI = entryBCI;
         this.id = id;
-        this.priority = priority;
         this.status = new AtomicReference<>(CompilationStatus.Queued);
     }
 
@@ -95,14 +92,6 @@
         return id;
     }
 
-    public int getPriority() {
-        return priority;
-    }
-
-    public boolean tryToCancel() {
-        return tryToChangeStatus(CompilationStatus.Queued, CompilationStatus.Canceled);
-    }
-
     public int getEntryBCI() {
         return entryBCI;
     }
@@ -110,12 +99,6 @@
     public void run() {
         withinEnqueue.set(Boolean.FALSE);
         try {
-            if (DynamicCompilePriority.getValue()) {
-                int threadPriority = priority < VMToCompilerImpl.SlowQueueCutoff.getValue() ? Thread.NORM_PRIORITY : Thread.MIN_PRIORITY;
-                if (Thread.currentThread().getPriority() != threadPriority) {
-                    Thread.currentThread().setPriority(threadPriority);
-                }
-            }
             runCompilation();
         } finally {
             if (method.currentTask() == this) {
@@ -131,12 +114,16 @@
     public static final DebugTimer CompilationTime = Debug.timer("CompilationTime");
 
     public void runCompilation() {
-        if (!tryToChangeStatus(CompilationStatus.Queued, CompilationStatus.Running) || method.hasCompiledCode()) {
-            return;
-        }
+        /*
+         * no code must be outside this try/finally because it could happen otherwise that
+         * clearQueuedForCompilation() is not executed
+         */
+        try (TimerCloseable a = CompilationTime.start()) {
+            if (!tryToChangeStatus(CompilationStatus.Queued, CompilationStatus.Running) || method.hasCompiledCode()) {
+                return;
+            }
 
-        CompilationStatistics stats = CompilationStatistics.create(method, entryBCI != StructuredGraph.INVOCATION_ENTRY_BCI);
-        try (TimerCloseable a = CompilationTime.start()) {
+            CompilationStatistics stats = CompilationStatistics.create(method, entryBCI != StructuredGraph.INVOCATION_ENTRY_BCI);
             final boolean printCompilation = PrintCompilation.getValue() && !TTY.isSuppressed();
             if (printCompilation) {
                 TTY.println(String.format("%-6d Graal %-70s %-45s %-50s %s...", id, method.getDeclaringClass().getName(), method.getName(), method.getSignature(),
@@ -178,6 +165,7 @@
             }
 
             installMethod(result);
+            stats.finish(method);
         } catch (BailoutException bailout) {
             Debug.metric("Bailouts").increment();
             if (ExitVMOnBailout.getValue()) {
@@ -199,7 +187,6 @@
             assert method.isQueuedForCompilation();
             method.clearQueuedForCompilation();
         }
-        stats.finish(method);
     }
 
     /**
@@ -234,16 +221,7 @@
     }
 
     @Override
-    public int compareTo(CompilationTask o) {
-        if (priority != o.priority) {
-            return priority - o.priority;
-        } else {
-            return id - o.id;
-        }
-    }
-
-    @Override
     public String toString() {
-        return "Compilation[id=" + id + ", prio=" + priority + " " + MetaUtil.format("%H.%n(%p)", method) + (entryBCI == StructuredGraph.INVOCATION_ENTRY_BCI ? "" : "@" + entryBCI) + "]";
+        return "Compilation[id=" + id + ", " + MetaUtil.format("%H.%n(%p)", method) + (entryBCI == StructuredGraph.INVOCATION_ENTRY_BCI ? "" : "@" + entryBCI) + "]";
     }
 }
--- a/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompileTheWorld.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompileTheWorld.java	Tue Jul 09 17:53:58 2013 +0200
@@ -218,7 +218,7 @@
     private void compileMethod(HotSpotResolvedJavaMethod method) {
         try {
             long start = System.currentTimeMillis();
-            vmToCompiler.compileMethod(method, StructuredGraph.INVOCATION_ENTRY_BCI, true, 10);
+            vmToCompiler.compileMethod(method, StructuredGraph.INVOCATION_ENTRY_BCI, true);
             compileTime += (System.currentTimeMillis() - start);
             compiledMethodsCounter++;
             method.reprofile();  // makes the method also not-entrant
--- a/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilerThread.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/CompilerThread.java	Tue Jul 09 17:53:58 2013 +0200
@@ -41,15 +41,6 @@
             return new CompilerThread(r);
         }
     };
-    public static final ThreadFactory LOW_PRIORITY_FACTORY = new ThreadFactory() {
-
-        @Override
-        public Thread newThread(Runnable r) {
-            CompilerThread thread = new CompilerThread(r);
-            thread.setPriority(MIN_PRIORITY);
-            return thread;
-        }
-    };
 
     private CompilerThread(Runnable r) {
         super(r);
--- a/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompiler.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompiler.java	Tue Jul 09 17:53:58 2013 +0200
@@ -38,12 +38,12 @@
      * Compiles a method to machine code. This method is called from the VM
      * (VMToCompiler::compileMethod).
      */
-    void compileMethod(long metaspaceMethod, HotSpotResolvedObjectType holder, int entryBCI, boolean blocking, int priority) throws Throwable;
+    void compileMethod(long metaspaceMethod, HotSpotResolvedObjectType holder, int entryBCI, boolean blocking) throws Throwable;
 
     /**
      * Compiles a method to machine code.
      */
-    void compileMethod(HotSpotResolvedJavaMethod method, int entryBCI, boolean blocking, int priority) throws Throwable;
+    void compileMethod(HotSpotResolvedJavaMethod method, int entryBCI, boolean blocking) throws Throwable;
 
     void shutdownCompiler() throws Throwable;
 
--- a/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompilerImpl.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.hotspot/src/com/oracle/graal/hotspot/bridge/VMToCompilerImpl.java	Tue Jul 09 17:53:58 2013 +0200
@@ -64,12 +64,6 @@
     @Option(help = "File to which compiler logging is sent")
     private static final OptionValue<String> LogFile = new OptionValue<>(null);
 
-    @Option(help = "Use low priority compilation threads")
-    private static final OptionValue<Boolean> SlowCompileThreads = new OptionValue<>(false);
-
-    @Option(help = "Use priority-based compilation queue")
-    private static final OptionValue<Boolean> PriorityCompileQueue = new OptionValue<>(true);
-
     @Option(help = "Print compilation queue activity periodically")
     private static final OptionValue<Boolean> PrintQueue = new OptionValue<>(false);
 
@@ -108,7 +102,6 @@
     public final HotSpotResolvedPrimitiveType typeVoid;
 
     private ThreadPoolExecutor compileQueue;
-    private ThreadPoolExecutor slowCompileQueue;
     private AtomicInteger compileTaskIds = new AtomicInteger();
 
     private volatile boolean bootstrapRunning;
@@ -211,13 +204,7 @@
         }
 
         // Create compilation queue.
-        BlockingQueue<Runnable> queue = PriorityCompileQueue.getValue() ? new PriorityBlockingQueue<Runnable>() : new LinkedBlockingQueue<Runnable>();
-        compileQueue = new ThreadPoolExecutor(Threads.getValue(), Threads.getValue(), 0L, TimeUnit.MILLISECONDS, queue, CompilerThread.FACTORY);
-
-        if (SlowCompileThreads.getValue()) {
-            BlockingQueue<Runnable> slowQueue = PriorityCompileQueue.getValue() ? new PriorityBlockingQueue<Runnable>() : new LinkedBlockingQueue<Runnable>();
-            slowCompileQueue = new ThreadPoolExecutor(Threads.getValue(), Threads.getValue(), 0L, TimeUnit.MILLISECONDS, slowQueue, CompilerThread.LOW_PRIORITY_FACTORY);
-        }
+        compileQueue = new ThreadPoolExecutor(Threads.getValue(), Threads.getValue(), 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(), CompilerThread.FACTORY);
 
         // Create queue status printing thread.
         if (PrintQueue.getValue()) {
@@ -226,11 +213,7 @@
                 @Override
                 public void run() {
                     while (true) {
-                        if (slowCompileQueue == null) {
-                            TTY.println(compileQueue.toString());
-                        } else {
-                            TTY.println("fast: " + compileQueue.toString() + " slow: " + slowCompileQueue);
-                        }
+                        TTY.println(compileQueue.toString());
                         try {
                             Thread.sleep(1000);
                         } catch (InterruptedException e) {
@@ -395,14 +378,8 @@
                 try {
                     assert !CompilationTask.withinEnqueue.get();
                     CompilationTask.withinEnqueue.set(Boolean.TRUE);
-                    if (slowCompileQueue == null) {
-                        if (compileQueue.getCompletedTaskCount() >= Math.max(3, compileQueue.getTaskCount())) {
-                            break;
-                        }
-                    } else {
-                        if (compileQueue.getCompletedTaskCount() + slowCompileQueue.getCompletedTaskCount() >= Math.max(3, compileQueue.getTaskCount() + slowCompileQueue.getTaskCount())) {
-                            break;
-                        }
+                    if (compileQueue.getCompletedTaskCount() >= Math.max(3, compileQueue.getTaskCount())) {
+                        break;
                     }
                 } finally {
                     CompilationTask.withinEnqueue.set(Boolean.FALSE);
@@ -440,7 +417,7 @@
     private void enqueue(Method m) throws Throwable {
         JavaMethod javaMethod = graalRuntime.getRuntime().lookupJavaMethod(m);
         assert !Modifier.isAbstract(((HotSpotResolvedJavaMethod) javaMethod).getModifiers()) && !Modifier.isNative(((HotSpotResolvedJavaMethod) javaMethod).getModifiers()) : javaMethod;
-        compileMethod((HotSpotResolvedJavaMethod) javaMethod, StructuredGraph.INVOCATION_ENTRY_BCI, false, 10);
+        compileMethod((HotSpotResolvedJavaMethod) javaMethod, StructuredGraph.INVOCATION_ENTRY_BCI, false);
     }
 
     private static void shutdownCompileQueue(ThreadPoolExecutor queue) throws InterruptedException {
@@ -458,7 +435,6 @@
             assert !CompilationTask.withinEnqueue.get();
             CompilationTask.withinEnqueue.set(Boolean.TRUE);
             shutdownCompileQueue(compileQueue);
-            shutdownCompileQueue(slowCompileQueue);
         } finally {
             CompilationTask.withinEnqueue.set(Boolean.FALSE);
         }
@@ -576,15 +552,15 @@
     }
 
     @Override
-    public void compileMethod(long metaspaceMethod, final HotSpotResolvedObjectType holder, final int entryBCI, boolean blocking, int priority) throws Throwable {
+    public void compileMethod(long metaspaceMethod, final HotSpotResolvedObjectType holder, final int entryBCI, boolean blocking) throws Throwable {
         HotSpotResolvedJavaMethod method = holder.createMethod(metaspaceMethod);
-        compileMethod(method, entryBCI, blocking, priority);
+        compileMethod(method, entryBCI, blocking);
     }
 
     /**
      * Compiles a method to machine code.
      */
-    public void compileMethod(final HotSpotResolvedJavaMethod method, final int entryBCI, boolean blocking, int priority) throws Throwable {
+    public void compileMethod(final HotSpotResolvedJavaMethod method, final int entryBCI, boolean blocking) throws Throwable {
         boolean osrCompilation = entryBCI != StructuredGraph.INVOCATION_ENTRY_BCI;
         if (osrCompilation && bootstrapRunning) {
             // no OSR compilations during bootstrap - the compiler is just too slow at this point,
@@ -600,39 +576,24 @@
             return;
         }
 
+        CompilationTask.withinEnqueue.set(Boolean.TRUE);
         try {
-            CompilationTask.withinEnqueue.set(Boolean.TRUE);
-            if (!method.tryToQueueForCompilation()) {
-                // method is already queued
-                CompilationTask current = method.currentTask();
-                if (!PriorityCompileQueue.getValue() || current == null || !current.tryToCancel()) {
-                    // normally compilation tasks will only be re-queued when they get a
-                    // priority boost, so cancel the old task and add a new one
-                    // without a prioritizing compile queue it makes no sense to re-queue the
-                    // compilation task
-                    return;
-                }
-            }
-            assert method.isQueuedForCompilation();
+            if (method.tryToQueueForCompilation()) {
+                assert method.isQueuedForCompilation();
+
+                final OptimisticOptimizations optimisticOpts = new OptimisticOptimizations(method);
+                int id = compileTaskIds.incrementAndGet();
+                CompilationTask task = CompilationTask.create(graalRuntime, createPhasePlan(optimisticOpts, osrCompilation), optimisticOpts, method, entryBCI, id);
 
-            final OptimisticOptimizations optimisticOpts = new OptimisticOptimizations(method);
-            int id = compileTaskIds.incrementAndGet();
-            // OSR compilations need to be finished quickly, so they get max priority
-            int queuePriority = osrCompilation ? -1 : priority;
-            CompilationTask task = CompilationTask.create(graalRuntime, createPhasePlan(optimisticOpts, osrCompilation), optimisticOpts, method, entryBCI, id, queuePriority);
-
-            if (blocking) {
-                task.runCompilation();
-            } else {
-                try {
-                    method.setCurrentTask(task);
-                    if (SlowCompileThreads.getValue() && priority > SlowQueueCutoff.getValue()) {
-                        slowCompileQueue.execute(task);
-                    } else {
+                if (blocking) {
+                    task.runCompilation();
+                } else {
+                    try {
+                        method.setCurrentTask(task);
                         compileQueue.execute(task);
+                    } catch (RejectedExecutionException e) {
+                        // The compile queue was already shut down.
                     }
-                } catch (RejectedExecutionException e) {
-                    // The compile queue was already shut down.
                 }
             }
         } finally {
--- a/graal/com.oracle.graal.phases/src/com/oracle/graal/phases/GraalOptions.java	Tue Jul 09 11:21:06 2013 +0200
+++ b/graal/com.oracle.graal.phases/src/com/oracle/graal/phases/GraalOptions.java	Tue Jul 09 17:53:58 2013 +0200
@@ -89,8 +89,6 @@
 
     // comilation queue
     @Option(help = "")
-    public static final OptionValue<Boolean> DynamicCompilePriority = new OptionValue<>(false);
-    @Option(help = "")
     public static final OptionValue<String> CompileTheWorld = new OptionValue<>(null);
     @Option(help = "")
     public static final OptionValue<Integer> CompileTheWorldStartAt = new OptionValue<>(1);
--- a/src/cpu/x86/vm/templateInterpreter_x86_64.cpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/cpu/x86/vm/templateInterpreter_x86_64.cpp	Tue Jul 09 17:53:58 2013 +0200
@@ -304,18 +304,6 @@
 // Helpers for commoning out cases in the various type of method entries.
 //
 
-#ifdef GRAALVM
-
-void graal_initialize_time(JavaThread* thread) {
-  assert(ProfileInterpreter, "must be profiling interpreter");
-  frame fr = thread->last_frame();
-  assert(fr.is_interpreted_frame(), "must come from interpreter");
-  assert(fr.interpreter_frame_method()->method_counters() != NULL, "need to initialize method counters");
-  fr.interpreter_frame_method()->method_counters()->set_graal_invocation_time(os::javaTimeNanos());
-}
-
-#endif // GRAALVM
-
 // increment invocation count & check for overflow
 //
 // Note: checking for negative value instead of overflow
@@ -369,30 +357,6 @@
               MethodCounters::interpreter_invocation_counter_offset()));
     }
 
-#ifdef GRAALVM
-    if (CompilationPolicyChoice == 4) {
-      Label not_zero;
-      __ testl(rcx, InvocationCounter::count_mask_value);
-      __ jcc(Assembler::notZero, not_zero);
-
-      __ push(rax);
-      __ push(rcx);
-      __ call_VM(noreg, CAST_FROM_FN_PTR(address, graal_initialize_time), rdx, false);
-      __ set_method_data_pointer_for_bcp();
-      __ get_method(rbx);
-      __ pop(rcx);
-      __ pop(rax);
-
-#ifdef ASSERT
-      __ testl(rcx, InvocationCounter::count_mask_value);
-      __ jcc(Assembler::zero, not_zero);
-      __ stop("unexpected counter value in rcx");
-#endif
-
-      __ bind(not_zero);
-    }
-#endif // GRAALVM
-
     // Update standard invocation counters
     __ movl(rcx, invocation_counter);
     __ incrementl(rcx, InvocationCounter::count_increment);
--- a/src/share/vm/classfile/vmSymbols.hpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/classfile/vmSymbols.hpp	Tue Jul 09 17:53:58 2013 +0200
@@ -355,7 +355,7 @@
   template(bootstrap_name,                        "bootstrap")                                                                        \
   template(shutdownCompiler_name,                 "shutdownCompiler")                                                                 \
   template(compileMethod_name,                    "compileMethod")                                                                    \
-  template(compileMethod_signature,               "(JLcom/oracle/graal/hotspot/meta/HotSpotResolvedObjectType;IZI)V")                 \
+  template(compileMethod_signature,               "(JLcom/oracle/graal/hotspot/meta/HotSpotResolvedObjectType;IZ)V")                  \
   template(setOption_name,                        "setOption")                                                                        \
   template(setOption_signature,                   "(Ljava/lang/String;)Z")                                                            \
   template(createUnresolvedJavaMethod_name,       "createUnresolvedJavaMethod")                                                       \
--- a/src/share/vm/graal/graalCompiler.cpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/graal/graalCompiler.cpp	Tue Jul 09 17:53:58 2013 +0200
@@ -171,8 +171,7 @@
   ResourceMark rm;
   JavaThread::current()->set_is_compiling(true);
   Handle holder = GraalCompiler::createHotSpotResolvedObjectType(method, CHECK);
-  int priority = GRAALVM_ONLY(method->method_counters()->graal_priority()) NOT_GRAALVM(0);
-  VMToCompiler::compileMethod(method(), holder, entry_bci, blocking, priority);
+  VMToCompiler::compileMethod(method(), holder, entry_bci, blocking);
   JavaThread::current()->set_is_compiling(false);
 }
 
--- a/src/share/vm/graal/graalVMToCompiler.cpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/graal/graalVMToCompiler.cpp	Tue Jul 09 17:53:58 2013 +0200
@@ -106,7 +106,7 @@
   return result.get_jboolean();
 }
 
-void VMToCompiler::compileMethod(Method* method, Handle holder, int entry_bci, jboolean blocking, int priority) {
+void VMToCompiler::compileMethod(Method* method, Handle holder, int entry_bci, jboolean blocking) {
   assert(method != NULL, "just checking");
   assert(!holder.is_null(), "just checking");
   Thread* THREAD = Thread::current();
@@ -117,7 +117,6 @@
   args.push_oop(holder());
   args.push_int(entry_bci);
   args.push_int(blocking);
-  args.push_int(priority);
   JavaCalls::call_interface(&result, vmToCompilerKlass(), vmSymbols::compileMethod_name(), vmSymbols::compileMethod_signature(), &args, THREAD);
   check_pending_exception("Error while calling compileMethod");
 }
--- a/src/share/vm/graal/graalVMToCompiler.hpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/graal/graalVMToCompiler.hpp	Tue Jul 09 17:53:58 2013 +0200
@@ -58,7 +58,7 @@
   static jboolean setOption(Handle option);
 
   // public abstract boolean compileMethod(long vmId, String name, int entry_bci, boolean blocking);
-  static void compileMethod(Method* method, Handle holder, int entry_bci, jboolean blocking, int priority);
+  static void compileMethod(Method* method, Handle holder, int entry_bci, jboolean blocking);
 
   // public abstract void shutdownCompiler();
   static void shutdownCompiler();
--- a/src/share/vm/oops/methodCounters.cpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/oops/methodCounters.cpp	Tue Jul 09 17:53:58 2013 +0200
@@ -32,10 +32,6 @@
 void MethodCounters::clear_counters() {
   invocation_counter()->reset();
   backedge_counter()->reset();
-#ifdef GRAALVM
-  set_graal_priority(0);
-  set_graal_invocation_time(0L);
-#endif
   set_interpreter_throwout_count(0);
   set_interpreter_invocation_count(0);
 }
--- a/src/share/vm/oops/methodCounters.hpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/oops/methodCounters.hpp	Tue Jul 09 17:53:58 2013 +0200
@@ -37,10 +37,6 @@
   InvocationCounter _invocation_counter;         // Incremented before each activation of the method - used to trigger frequency-based optimizations
   InvocationCounter _backedge_counter;           // Incremented before each backedge taken - used to trigger frequencey-based optimizations
 
-#ifdef GRAALVM
-  jlong             _graal_invocation_time;
-  int               _graal_priority;
-#endif
 #ifdef TIERED
   float             _rate;                        // Events (invocation and backedge counter increments) per millisecond
   jlong             _prev_time;                   // Previous time the rate was acquired
@@ -49,10 +45,6 @@
   MethodCounters() : _interpreter_invocation_count(0),
                      _interpreter_throwout_count(0),
                      _number_of_breakpoints(0)
-#ifdef GRAALVM
-                     , _graal_invocation_time(0L)
-                     , _graal_priority(0)
-#endif
 
 #ifdef TIERED
                    , _rate(0),
@@ -102,15 +94,6 @@
   void decr_number_of_breakpoints()    { --_number_of_breakpoints; }
   void clear_number_of_breakpoints()   { _number_of_breakpoints = 0; }
 
-#ifdef GRAALVM
-  void set_graal_invocation_time(jlong time) { _graal_invocation_time = time; }
-  jlong graal_invocation_time()              { return _graal_invocation_time; }
-
-  void set_graal_priority(int prio)          { _graal_priority = prio; }
-  int graal_priority()                       { return _graal_priority; }
-#endif // GRAAL
-
-
 #ifdef TIERED
   jlong prev_time() const                        { return _prev_time; }
   void set_prev_time(jlong time)                 { _prev_time = time; }
--- a/src/share/vm/runtime/compilationPolicy.cpp	Tue Jul 09 11:21:06 2013 +0200
+++ b/src/share/vm/runtime/compilationPolicy.cpp	Tue Jul 09 17:53:58 2013 +0200
@@ -80,15 +80,8 @@
     Unimplemented();
 #endif
     break;
-  case 4:
-#ifdef GRAALVM
-    CompilationPolicy::set_policy(new GraalCompPolicy());
-#else
-    Unimplemented();
-#endif
-    break;
   default:
-    fatal("CompilationPolicyChoice must be in the range: [0-4]");
+    fatal("CompilationPolicyChoice must be in the range: [0-3]");
   }
   CompilationPolicy::policy()->initialize();
 }
@@ -471,79 +464,6 @@
   }
 }
 
-// GraalCompPolicy - compile current method
-
-#ifdef GRAALVM
-
-void GraalCompPolicy::method_invocation_event(methodHandle m, JavaThread* thread) {
-  MethodCounters* mcs = m->method_counters();
-  assert(mcs != NULL, "method counters should be initialized");
-  int hot_count = m->invocation_count();
-  jlong hot_time = mcs->graal_invocation_time();
-  reset_counter_for_invocation_event(m);
-
-  if (is_compilation_enabled() && can_be_compiled(m)) {
-    nmethod* nm = m->code();
-    if (nm == NULL) {
-      if (hot_count > 1) {
-        jlong current_time = os::javaTimeNanos();
-        int time_per_call = (int) ((current_time - hot_time) / hot_count);
-        if (mcs != NULL) {
-          mcs->set_graal_invocation_time(current_time);
-        }
-        if (UseNewCode) {
-          if (m->queued_for_compilation()) {
-            if (time_per_call < (mcs->graal_priority() / 5)) {
-              mcs->set_graal_priority(time_per_call);
-              m->clear_queued_for_compilation();
-            }
-          } else {
-            if (time_per_call < mcs->graal_priority()) {
-              mcs->set_graal_priority(time_per_call);
-            }
-          }
-        } else {
-          if (time_per_call < mcs->graal_priority()) {
-            mcs->set_graal_priority(time_per_call);
-          }
-        }
-      }
-
-      if (!m->queued_for_compilation()) {
-        if (TraceCompilationPolicy) {
-          tty->print("method invocation trigger: ");
-          m->print_short_name(tty);
-          tty->print_cr(" ( interpreted " INTPTR_FORMAT ", size=%d, hotCount=%d, hotTime=" UINT64_FORMAT " ) ", (address)m(), m->code_size(), hot_count, hot_time);
-        }
-
-        assert(m->is_native() || m->method_data() != NULL, "do not compile code methods");
-        CompileBroker::compile_method(m, InvocationEntryBci, CompLevel_highest_tier, m, hot_count, "count", thread);
-      }
-    }
-  }
-}
-
-void GraalCompPolicy::method_back_branch_event(methodHandle m, int bci, JavaThread* thread) {
-  int hot_count = m->backedge_count();
-  const char* comment = "backedge_count";
-  reset_counter_for_back_branch_event(m);
-
-  if (is_compilation_enabled() && !m->is_not_osr_compilable() && can_be_compiled(m) && !m->queued_for_compilation() && m->code() == NULL) {
-    if (TraceCompilationPolicy) {
-      tty->print("backedge invocation trigger: ");
-      m->print_short_name(tty);
-      tty->print_cr(" ( interpreted " INTPTR_FORMAT ", size=%d, hotCount=%d ) ", (address)m(), m->code_size(), hot_count);
-    }
-
-    CompileBroker::compile_method(m, bci, CompLevel_highest_tier,
-                                  m, hot_count, comment, thread);
-    NOT_PRODUCT(trace_osr_completion(m->lookup_osr_nmethod_for(bci, CompLevel_highest_tier, true));)
-  }
-}
-
-#endif // GRAALVM
-
-
 // StackWalkCompPolicy - walk up stack to find a suitable method to compile
 
 #ifdef COMPILER2