changeset 5484:469216acdb28

8023014: CodeSweeperSweepNoFlushTest.java fails with HS crash Summary: Ensure ensure correct initialization of compiler runtime Reviewed-by: kvn, twisti
author anoll
date Thu, 10 Oct 2013 15:44:12 +0200
parents 8b80b262e501
children ed2c74787eb5
files src/share/vm/c1/c1_Compiler.cpp src/share/vm/c1/c1_Compiler.hpp src/share/vm/code/codeBlob.cpp src/share/vm/code/codeBlob.hpp src/share/vm/compiler/abstractCompiler.cpp src/share/vm/compiler/abstractCompiler.hpp src/share/vm/compiler/compileBroker.cpp src/share/vm/compiler/compileBroker.hpp src/share/vm/opto/c2compiler.cpp src/share/vm/opto/c2compiler.hpp src/share/vm/opto/runtime.cpp src/share/vm/opto/runtime.hpp src/share/vm/runtime/thread.cpp src/share/vm/runtime/thread.hpp src/share/vm/runtime/vmStructs.cpp src/share/vm/shark/sharkCompiler.cpp src/share/vm/shark/sharkCompiler.hpp test/compiler/startup/SmallCodeCacheStartup.java
diffstat 18 files changed, 384 insertions(+), 263 deletions(-) [+]
line wrap: on
line diff
--- a/src/share/vm/c1/c1_Compiler.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/c1/c1_Compiler.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -42,26 +42,16 @@
 #include "runtime/interfaceSupport.hpp"
 #include "runtime/sharedRuntime.hpp"
 
-volatile int Compiler::_runtimes = uninitialized;
-
-Compiler::Compiler() {
-}
-
 
-Compiler::~Compiler() {
-  Unimplemented();
-}
+Compiler::Compiler () {}
 
-
-void Compiler::initialize_all() {
+void Compiler::init_c1_runtime() {
   BufferBlob* buffer_blob = CompilerThread::current()->get_buffer_blob();
   Arena* arena = new (mtCompiler) Arena();
   Runtime1::initialize(buffer_blob);
   FrameMap::initialize();
   // initialize data structures
   ValueType::initialize(arena);
-  // Instruction::initialize();
-  // BlockBegin::initialize();
   GraphBuilder::initialize();
   // note: to use more than one instance of LinearScan at a time this function call has to
   //       be moved somewhere outside of this constructor:
@@ -70,32 +60,33 @@
 
 
 void Compiler::initialize() {
-  if (_runtimes != initialized) {
-    initialize_runtimes( initialize_all, &_runtimes);
+  // Buffer blob must be allocated per C1 compiler thread at startup
+  BufferBlob* buffer_blob = init_buffer_blob();
+
+  if (should_perform_init()) {
+    if (buffer_blob == NULL) {
+      // When we come here we are in state 'initializing'; entire C1 compilation
+      // can be shut down.
+      set_state(failed);
+    } else {
+      init_c1_runtime();
+      set_state(initialized);
+    }
   }
-  mark_initialized();
 }
 
-
-BufferBlob* Compiler::get_buffer_blob(ciEnv* env) {
+BufferBlob* Compiler::init_buffer_blob() {
   // Allocate buffer blob once at startup since allocation for each
   // compilation seems to be too expensive (at least on Intel win32).
-  BufferBlob* buffer_blob = CompilerThread::current()->get_buffer_blob();
-  if (buffer_blob != NULL) {
-    return buffer_blob;
-  }
+  assert (CompilerThread::current()->get_buffer_blob() == NULL, "Should initialize only once");
 
   // setup CodeBuffer.  Preallocate a BufferBlob of size
   // NMethodSizeLimit plus some extra space for constants.
   int code_buffer_size = Compilation::desired_max_code_buffer_size() +
     Compilation::desired_max_constant_size();
 
-  buffer_blob = BufferBlob::create("Compiler1 temporary CodeBuffer",
-                                   code_buffer_size);
-  if (buffer_blob == NULL) {
-    CompileBroker::handle_full_code_cache();
-    env->record_failure("CodeCache is full");
-  } else {
+  BufferBlob* buffer_blob = BufferBlob::create("C1 temporary CodeBuffer", code_buffer_size);
+  if (buffer_blob != NULL) {
     CompilerThread::current()->set_buffer_blob(buffer_blob);
   }
 
@@ -104,15 +95,8 @@
 
 
 void Compiler::compile_method(ciEnv* env, ciMethod* method, int entry_bci) {
-  BufferBlob* buffer_blob = Compiler::get_buffer_blob(env);
-  if (buffer_blob == NULL) {
-    return;
-  }
-
-  if (!is_initialized()) {
-    initialize();
-  }
-
+  BufferBlob* buffer_blob = CompilerThread::current()->get_buffer_blob();
+  assert(buffer_blob != NULL, "Must exist");
   // invoke compilation
   {
     // We are nested here because we need for the destructor
--- a/src/share/vm/c1/c1_Compiler.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/c1/c1_Compiler.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -30,11 +30,9 @@
 // There is one instance of the Compiler per CompilerThread.
 
 class Compiler: public AbstractCompiler {
-
  private:
-
- // Tracks whether runtime has been initialized
- static volatile int _runtimes;
+  static void init_c1_runtime();
+  BufferBlob* init_buffer_blob();
 
  public:
   // Creation
@@ -46,19 +44,12 @@
 
   virtual bool is_c1()                           { return true; };
 
-  BufferBlob* get_buffer_blob(ciEnv* env);
-
   // Missing feature tests
   virtual bool supports_native()                 { return true; }
   virtual bool supports_osr   ()                 { return true; }
 
-  // Customization
-  virtual bool needs_adapters         ()         { return false; }
-  virtual bool needs_stubs            ()         { return false; }
-
   // Initialization
   virtual void initialize();
-  static  void initialize_all();
 
   // Compilation entry point for methods
   virtual void compile_method(ciEnv* env, ciMethod* target, int entry_bci);
--- a/src/share/vm/code/codeBlob.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/code/codeBlob.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -245,8 +245,8 @@
 }
 
 
-void* BufferBlob::operator new(size_t s, unsigned size) throw() {
-  void* p = CodeCache::allocate(size);
+void* BufferBlob::operator new(size_t s, unsigned size, bool is_critical) throw() {
+  void* p = CodeCache::allocate(size, is_critical);
   return p;
 }
 
@@ -277,7 +277,10 @@
   unsigned int size = allocation_size(cb, sizeof(AdapterBlob));
   {
     MutexLockerEx mu(CodeCache_lock, Mutex::_no_safepoint_check_flag);
-    blob = new (size) AdapterBlob(size, cb);
+    // The parameter 'true' indicates a critical memory allocation.
+    // This means that CodeCacheMinimumFreeSpace is used, if necessary
+    const bool is_critical = true;
+    blob = new (size, is_critical) AdapterBlob(size, cb);
   }
   // Track memory usage statistic after releasing CodeCache_lock
   MemoryService::track_code_cache_memory_usage();
@@ -299,7 +302,10 @@
   size += round_to(buffer_size, oopSize);
   {
     MutexLockerEx mu(CodeCache_lock, Mutex::_no_safepoint_check_flag);
-    blob = new (size) MethodHandlesAdapterBlob(size);
+    // The parameter 'true' indicates a critical memory allocation.
+    // This means that CodeCacheMinimumFreeSpace is used, if necessary
+    const bool is_critical = true;
+    blob = new (size, is_critical) MethodHandlesAdapterBlob(size);
   }
   // Track memory usage statistic after releasing CodeCache_lock
   MemoryService::track_code_cache_memory_usage();
--- a/src/share/vm/code/codeBlob.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/code/codeBlob.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -209,7 +209,7 @@
   BufferBlob(const char* name, int size);
   BufferBlob(const char* name, int size, CodeBuffer* cb);
 
-  void* operator new(size_t s, unsigned size) throw();
+  void* operator new(size_t s, unsigned size, bool is_critical = false) throw();
 
  public:
   // Creation
@@ -253,7 +253,6 @@
 class MethodHandlesAdapterBlob: public BufferBlob {
 private:
   MethodHandlesAdapterBlob(int size)                 : BufferBlob("MethodHandles adapters", size) {}
-  MethodHandlesAdapterBlob(int size, CodeBuffer* cb) : BufferBlob("MethodHandles adapters", size, cb) {}
 
 public:
   // Creation
--- a/src/share/vm/compiler/abstractCompiler.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/compiler/abstractCompiler.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -24,41 +24,42 @@
 
 #include "precompiled.hpp"
 #include "compiler/abstractCompiler.hpp"
+#include "compiler/compileBroker.hpp"
 #include "runtime/mutexLocker.hpp"
-void AbstractCompiler::initialize_runtimes(initializer f, volatile int* state) {
-  if (*state != initialized) {
+
+bool AbstractCompiler::should_perform_init() {
+  if (_compiler_state != initialized) {
+    MutexLocker only_one(CompileThread_lock);
 
-    // We are thread in native here...
-    CompilerThread* thread = CompilerThread::current();
-    bool do_initialization = false;
-    {
-      ThreadInVMfromNative tv(thread);
-      ResetNoHandleMark rnhm;
-      MutexLocker only_one(CompileThread_lock, thread);
-      if ( *state == uninitialized) {
-        do_initialization = true;
-        *state = initializing;
-      } else {
-        while (*state == initializing ) {
-          CompileThread_lock->wait();
-        }
+    if (_compiler_state == uninitialized) {
+      _compiler_state = initializing;
+      return true;
+    } else {
+      while (_compiler_state == initializing) {
+        CompileThread_lock->wait();
       }
     }
-    if (do_initialization) {
-      // We can not hold any locks here since JVMTI events may call agents
+  }
+  return false;
+}
 
-      // Compiler(s) run as native
-
-      (*f)();
-
-      // To in_vm so we can use the lock
+bool AbstractCompiler::should_perform_shutdown() {
+  // Since this method can be called by multiple threads, the lock ensures atomicity of
+  // decrementing '_num_compiler_threads' and the following operations.
+  MutexLocker only_one(CompileThread_lock);
+  _num_compiler_threads--;
+  assert (CompileBroker::is_compilation_disabled_forever(), "Must be set, otherwise thread waits forever");
 
-      ThreadInVMfromNative tv(thread);
-      ResetNoHandleMark rnhm;
-      MutexLocker only_one(CompileThread_lock, thread);
-      assert(*state == initializing, "wrong state");
-      *state = initialized;
-      CompileThread_lock->notify_all();
-    }
+  // Only the last thread will perform shutdown operations
+  if (_num_compiler_threads == 0) {
+    return true;
   }
+  return false;
 }
+
+void AbstractCompiler::set_state(int state) {
+  // Ensure that ste is only set by one thread at a time
+  MutexLocker only_one(CompileThread_lock);
+  _compiler_state =  state;
+  CompileThread_lock->notify_all();
+}
--- a/src/share/vm/compiler/abstractCompiler.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/compiler/abstractCompiler.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -27,22 +27,25 @@
 
 #include "ci/compilerInterface.hpp"
 
-typedef void (*initializer)(void);
-
 class AbstractCompiler : public CHeapObj<mtCompiler> {
  private:
-  bool _is_initialized; // Mark whether compiler object is initialized
+  volatile int _num_compiler_threads;
 
  protected:
+  volatile int _compiler_state;
   // Used for tracking global state of compiler runtime initialization
-  enum { uninitialized, initializing, initialized };
+  enum { uninitialized, initializing, initialized, failed, shut_down };
 
-  // This method will call the initialization method "f" once (per compiler class/subclass)
-  // and do so without holding any locks
-  void initialize_runtimes(initializer f, volatile int* state);
+  // This method returns true for the first compiler thread that reaches that methods.
+  // This thread will initialize the compiler runtime.
+  bool should_perform_init();
 
  public:
-  AbstractCompiler() : _is_initialized(false)    {}
+  AbstractCompiler() : _compiler_state(uninitialized), _num_compiler_threads(0) {}
+
+  // This function determines the compiler thread that will perform the
+  // shutdown of the corresponding compiler runtime.
+  bool should_perform_shutdown();
 
   // Name of this compiler
   virtual const char* name() = 0;
@@ -74,17 +77,18 @@
 #endif // TIERED
 
   // Customization
-  virtual bool needs_stubs            ()         = 0;
+  virtual void initialize () = 0;
 
-  void mark_initialized()                        { _is_initialized = true; }
-  bool is_initialized()                          { return _is_initialized; }
+  void set_num_compiler_threads(int num) { _num_compiler_threads = num;  }
+  int num_compiler_threads()             { return _num_compiler_threads; }
 
-  virtual void initialize()                      = 0;
-
+  // Get/set state of compiler objects
+  bool is_initialized()           { return _compiler_state == initialized; }
+  bool is_failed     ()           { return _compiler_state == failed;}
+  void set_state     (int state);
+  void set_shut_down ()           { set_state(shut_down); }
   // Compilation entry point for methods
-  virtual void compile_method(ciEnv* env,
-                              ciMethod* target,
-                              int entry_bci) {
+  virtual void compile_method(ciEnv* env, ciMethod* target, int entry_bci) {
     ShouldNotReachHere();
   }
 
--- a/src/share/vm/compiler/compileBroker.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/compiler/compileBroker.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -186,7 +186,7 @@
 CompileQueue* CompileBroker::_c1_method_queue    = NULL;
 CompileTask*  CompileBroker::_task_free_list     = NULL;
 
-GrowableArray<CompilerThread*>* CompileBroker::_method_threads = NULL;
+GrowableArray<CompilerThread*>* CompileBroker::_compiler_threads = NULL;
 
 
 class CompilationLog : public StringEventLog {
@@ -587,9 +587,6 @@
 
 
 
-// ------------------------------------------------------------------
-// CompileQueue::add
-//
 // Add a CompileTask to a CompileQueue
 void CompileQueue::add(CompileTask* task) {
   assert(lock()->owned_by_self(), "must own lock");
@@ -626,6 +623,16 @@
   lock()->notify_all();
 }
 
+void CompileQueue::delete_all() {
+  assert(lock()->owned_by_self(), "must own lock");
+  if (_first != NULL) {
+    for (CompileTask* task = _first; task != NULL; task = task->next()) {
+      delete task;
+    }
+    _first = NULL;
+  }
+}
+
 // ------------------------------------------------------------------
 // CompileQueue::get
 //
@@ -640,6 +647,11 @@
   // case we perform code cache sweeps to free memory such that we can re-enable
   // compilation.
   while (_first == NULL) {
+    // Exit loop if compilation is disabled forever
+    if (CompileBroker::is_compilation_disabled_forever()) {
+      return NULL;
+    }
+
     if (UseCodeCacheFlushing && !CompileBroker::should_compile_new_jobs()) {
       // Wait a certain amount of time to possibly do another sweep.
       // We must wait until stack scanning has happened so that we can
@@ -664,9 +676,17 @@
       // remains unchanged. This behavior is desired, since we want to keep
       // the stable state, i.e., we do not want to evict methods from the
       // code cache if it is unnecessary.
-      lock()->wait();
+      // We need a timed wait here, since compiler threads can exit if compilation
+      // is disabled forever. We use 5 seconds wait time; the exiting of compiler threads
+      // is not critical and we do not want idle compiler threads to wake up too often.
+      lock()->wait(!Mutex::_no_safepoint_check_flag, 5*1000);
     }
   }
+
+  if (CompileBroker::is_compilation_disabled_forever()) {
+    return NULL;
+  }
+
   CompileTask* task = CompilationPolicy::policy()->select_task(this);
   remove(task);
   return task;
@@ -891,10 +911,8 @@
 }
 
 
-
-// ------------------------------------------------------------------
-// CompileBroker::make_compiler_thread
-CompilerThread* CompileBroker::make_compiler_thread(const char* name, CompileQueue* queue, CompilerCounters* counters, TRAPS) {
+CompilerThread* CompileBroker::make_compiler_thread(const char* name, CompileQueue* queue, CompilerCounters* counters,
+                                                    AbstractCompiler* comp, TRAPS) {
   CompilerThread* compiler_thread = NULL;
 
   Klass* k =
@@ -961,6 +979,7 @@
     java_lang_Thread::set_daemon(thread_oop());
 
     compiler_thread->set_threadObj(thread_oop());
+    compiler_thread->set_compiler(comp);
     Threads::add(compiler_thread);
     Thread::start(compiler_thread);
   }
@@ -972,25 +991,24 @@
 }
 
 
-// ------------------------------------------------------------------
-// CompileBroker::init_compiler_threads
-//
-// Initialize the compilation queue
 void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler_count) {
   EXCEPTION_MARK;
 #if !defined(ZERO) && !defined(SHARK)
   assert(c2_compiler_count > 0 || c1_compiler_count > 0, "No compilers?");
 #endif // !ZERO && !SHARK
+  // Initialize the compilation queue
   if (c2_compiler_count > 0) {
     _c2_method_queue  = new CompileQueue("C2MethodQueue",  MethodCompileQueue_lock);
+    _compilers[1]->set_num_compiler_threads(c2_compiler_count);
   }
   if (c1_compiler_count > 0) {
     _c1_method_queue  = new CompileQueue("C1MethodQueue",  MethodCompileQueue_lock);
+    _compilers[0]->set_num_compiler_threads(c1_compiler_count);
   }
 
   int compiler_count = c1_compiler_count + c2_compiler_count;
 
-  _method_threads =
+  _compiler_threads =
     new (ResourceObj::C_HEAP, mtCompiler) GrowableArray<CompilerThread*>(compiler_count, true);
 
   char name_buffer[256];
@@ -998,21 +1016,22 @@
     // Create a name for our thread.
     sprintf(name_buffer, "C2 CompilerThread%d", i);
     CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK);
-    CompilerThread* new_thread = make_compiler_thread(name_buffer, _c2_method_queue, counters, CHECK);
-    _method_threads->append(new_thread);
+    // Shark and C2
+    CompilerThread* new_thread = make_compiler_thread(name_buffer, _c2_method_queue, counters, _compilers[1], CHECK);
+    _compiler_threads->append(new_thread);
   }
 
   for (int i = c2_compiler_count; i < compiler_count; i++) {
     // Create a name for our thread.
     sprintf(name_buffer, "C1 CompilerThread%d", i);
     CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK);
-    CompilerThread* new_thread = make_compiler_thread(name_buffer, _c1_method_queue, counters, CHECK);
-    _method_threads->append(new_thread);
+    // C1
+    CompilerThread* new_thread = make_compiler_thread(name_buffer, _c1_method_queue, counters, _compilers[0], CHECK);
+    _compiler_threads->append(new_thread);
   }
 
   if (UsePerfData) {
-    PerfDataManager::create_constant(SUN_CI, "threads", PerfData::U_Bytes,
-                                     compiler_count, CHECK);
+    PerfDataManager::create_constant(SUN_CI, "threads", PerfData::U_Bytes, compiler_count, CHECK);
   }
 }
 
@@ -1029,27 +1048,6 @@
 }
 
 // ------------------------------------------------------------------
-// CompileBroker::is_idle
-bool CompileBroker::is_idle() {
-  if (_c2_method_queue != NULL && !_c2_method_queue->is_empty()) {
-    return false;
-  } else if (_c1_method_queue != NULL && !_c1_method_queue->is_empty()) {
-    return false;
-  } else {
-    int num_threads = _method_threads->length();
-    for (int i=0; i<num_threads; i++) {
-      if (_method_threads->at(i)->task() != NULL) {
-        return false;
-      }
-    }
-
-    // No pending or active compilations.
-    return true;
-  }
-}
-
-
-// ------------------------------------------------------------------
 // CompileBroker::compile_method
 //
 // Request compilation of a method.
@@ -1551,6 +1549,101 @@
   free_task(task);
 }
 
+// Initialize compiler thread(s) + compiler object(s). The postcondition
+// of this function is that the compiler runtimes are initialized and that
+//compiler threads can start compiling.
+bool CompileBroker::init_compiler_runtime() {
+  CompilerThread* thread = CompilerThread::current();
+  AbstractCompiler* comp = thread->compiler();
+  // Final sanity check - the compiler object must exist
+  guarantee(comp != NULL, "Compiler object must exist");
+
+  int system_dictionary_modification_counter;
+  {
+    MutexLocker locker(Compile_lock, thread);
+    system_dictionary_modification_counter = SystemDictionary::number_of_modifications();
+  }
+
+  {
+    // Must switch to native to allocate ci_env
+    ThreadToNativeFromVM ttn(thread);
+    ciEnv ci_env(NULL, system_dictionary_modification_counter);
+    // Cache Jvmti state
+    ci_env.cache_jvmti_state();
+    // Cache DTrace flags
+    ci_env.cache_dtrace_flags();
+
+    // Switch back to VM state to do compiler initialization
+    ThreadInVMfromNative tv(thread);
+    ResetNoHandleMark rnhm;
+
+
+    if (!comp->is_shark()) {
+      // Perform per-thread and global initializations
+      comp->initialize();
+    }
+  }
+
+  if (comp->is_failed()) {
+    disable_compilation_forever();
+    // If compiler initialization failed, no compiler thread that is specific to a
+    // particular compiler runtime will ever start to compile methods.
+
+    shutdown_compiler_runtime(comp, thread);
+    return false;
+  }
+
+  // C1 specific check
+  if (comp->is_c1() && (thread->get_buffer_blob() == NULL)) {
+    warning("Initialization of %s thread failed (no space to run compilers)", thread->name());
+    return false;
+  }
+
+  return true;
+}
+
+// If C1 and/or C2 initialization failed, we shut down all compilation.
+// We do this to keep things simple. This can be changed if it ever turns out to be
+// a problem.
+void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerThread* thread) {
+  // Free buffer blob, if allocated
+  if (thread->get_buffer_blob() != NULL) {
+    MutexLockerEx mu(CodeCache_lock, Mutex::_no_safepoint_check_flag);
+    CodeCache::free(thread->get_buffer_blob());
+  }
+
+  if (comp->should_perform_shutdown()) {
+    // There are two reasons for shutting down the compiler
+    // 1) compiler runtime initialization failed
+    // 2) The code cache is full and the following flag is set: -XX:-UseCodeCacheFlushing
+    warning("Shutting down compiler %s (no space to run compilers)", comp->name());
+
+    // Only one thread per compiler runtime object enters here
+    // Set state to shut down
+    comp->set_shut_down();
+
+    MutexLocker mu(MethodCompileQueue_lock, thread);
+    CompileQueue* queue;
+    if (_c1_method_queue != NULL) {
+      _c1_method_queue->delete_all();
+      queue = _c1_method_queue;
+      _c1_method_queue = NULL;
+      delete _c1_method_queue;
+    }
+
+    if (_c2_method_queue != NULL) {
+      _c2_method_queue->delete_all();
+      queue = _c2_method_queue;
+      _c2_method_queue = NULL;
+      delete _c2_method_queue;
+    }
+
+    // We could delete compiler runtimes also. However, there are references to
+    // the compiler runtime(s) (e.g.,  nmethod::is_compiled_by_c1()) which then
+    // fail. This can be done later if necessary.
+  }
+}
+
 // ------------------------------------------------------------------
 // CompileBroker::compiler_thread_loop
 //
@@ -1558,7 +1651,6 @@
 void CompileBroker::compiler_thread_loop() {
   CompilerThread* thread = CompilerThread::current();
   CompileQueue* queue = thread->queue();
-
   // For the thread that initializes the ciObjectFactory
   // this resource mark holds all the shared objects
   ResourceMark rm;
@@ -1587,65 +1679,78 @@
     log->end_elem();
   }
 
-  while (true) {
-    {
-      // We need this HandleMark to avoid leaking VM handles.
-      HandleMark hm(thread);
+  // If compiler thread/runtime initialization fails, exit the compiler thread
+  if (!init_compiler_runtime()) {
+    return;
+  }
 
-      if (CodeCache::unallocated_capacity() < CodeCacheMinimumFreeSpace) {
-        // the code cache is really full
-        handle_full_code_cache();
-      }
+  // Poll for new compilation tasks as long as the JVM runs. Compilation
+  // should only be disabled if something went wrong while initializing the
+  // compiler runtimes. This, in turn, should not happen. The only known case
+  // when compiler runtime initialization fails is if there is not enough free
+  // space in the code cache to generate the necessary stubs, etc.
+  while (!is_compilation_disabled_forever()) {
+    // We need this HandleMark to avoid leaking VM handles.
+    HandleMark hm(thread);
 
-      CompileTask* task = queue->get();
+    if (CodeCache::unallocated_capacity() < CodeCacheMinimumFreeSpace) {
+      // the code cache is really full
+      handle_full_code_cache();
+    }
 
-      // Give compiler threads an extra quanta.  They tend to be bursty and
-      // this helps the compiler to finish up the job.
-      if( CompilerThreadHintNoPreempt )
-        os::hint_no_preempt();
+    CompileTask* task = queue->get();
+    if (task == NULL) {
+      continue;
+    }
 
-      // trace per thread time and compile statistics
-      CompilerCounters* counters = ((CompilerThread*)thread)->counters();
-      PerfTraceTimedEvent(counters->time_counter(), counters->compile_counter());
+    // Give compiler threads an extra quanta.  They tend to be bursty and
+    // this helps the compiler to finish up the job.
+    if( CompilerThreadHintNoPreempt )
+      os::hint_no_preempt();
 
-      // Assign the task to the current thread.  Mark this compilation
-      // thread as active for the profiler.
-      CompileTaskWrapper ctw(task);
-      nmethodLocker result_handle;  // (handle for the nmethod produced by this task)
-      task->set_code_handle(&result_handle);
-      methodHandle method(thread, task->method());
+    // trace per thread time and compile statistics
+    CompilerCounters* counters = ((CompilerThread*)thread)->counters();
+    PerfTraceTimedEvent(counters->time_counter(), counters->compile_counter());
 
-      // Never compile a method if breakpoints are present in it
-      if (method()->number_of_breakpoints() == 0) {
-        // Compile the method.
-        if ((UseCompiler || AlwaysCompileLoopMethods) && CompileBroker::should_compile_new_jobs()) {
+    // Assign the task to the current thread.  Mark this compilation
+    // thread as active for the profiler.
+    CompileTaskWrapper ctw(task);
+    nmethodLocker result_handle;  // (handle for the nmethod produced by this task)
+    task->set_code_handle(&result_handle);
+    methodHandle method(thread, task->method());
+
+    // Never compile a method if breakpoints are present in it
+    if (method()->number_of_breakpoints() == 0) {
+      // Compile the method.
+      if ((UseCompiler || AlwaysCompileLoopMethods) && CompileBroker::should_compile_new_jobs()) {
 #ifdef COMPILER1
-          // Allow repeating compilations for the purpose of benchmarking
-          // compile speed. This is not useful for customers.
-          if (CompilationRepeat != 0) {
-            int compile_count = CompilationRepeat;
-            while (compile_count > 0) {
-              invoke_compiler_on_method(task);
-              nmethod* nm = method->code();
-              if (nm != NULL) {
-                nm->make_zombie();
-                method->clear_code();
-              }
-              compile_count--;
+        // Allow repeating compilations for the purpose of benchmarking
+        // compile speed. This is not useful for customers.
+        if (CompilationRepeat != 0) {
+          int compile_count = CompilationRepeat;
+          while (compile_count > 0) {
+            invoke_compiler_on_method(task);
+            nmethod* nm = method->code();
+            if (nm != NULL) {
+              nm->make_zombie();
+              method->clear_code();
             }
+            compile_count--;
           }
+        }
 #endif /* COMPILER1 */
-          invoke_compiler_on_method(task);
-        } else {
-          // After compilation is disabled, remove remaining methods from queue
-          method->clear_queued_for_compilation();
-        }
+        invoke_compiler_on_method(task);
+      } else {
+        // After compilation is disabled, remove remaining methods from queue
+        method->clear_queued_for_compilation();
       }
     }
   }
+
+  // Shut down compiler runtime
+  shutdown_compiler_runtime(thread->compiler(), thread);
 }
 
-
 // ------------------------------------------------------------------
 // CompileBroker::init_compiler_thread_log
 //
@@ -1960,8 +2065,7 @@
         NMethodSweeper::possibly_sweep();
       }
     } else {
-      UseCompiler               = false;
-      AlwaysCompileLoopMethods  = false;
+      disable_compilation_forever();
     }
   }
   codecache_print(/* detailed= */ true);
--- a/src/share/vm/compiler/compileBroker.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/compiler/compileBroker.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -213,8 +213,12 @@
 
   // Redefine Classes support
   void mark_on_stack();
+  void delete_all();
+  void         print();
 
-  void         print();
+  ~CompileQueue() {
+    assert (is_empty(), " Compile Queue must be empty");
+  }
 };
 
 // CompileTaskWrapper
@@ -266,7 +270,7 @@
   static CompileQueue* _c1_method_queue;
   static CompileTask* _task_free_list;
 
-  static GrowableArray<CompilerThread*>* _method_threads;
+  static GrowableArray<CompilerThread*>* _compiler_threads;
 
   // performance counters
   static PerfCounter* _perf_total_compilation;
@@ -311,7 +315,7 @@
   static int _sum_nmethod_code_size;
   static long _peak_compilation_time;
 
-  static CompilerThread* make_compiler_thread(const char* name, CompileQueue* queue, CompilerCounters* counters, TRAPS);
+  static CompilerThread* make_compiler_thread(const char* name, CompileQueue* queue, CompilerCounters* counters, AbstractCompiler* comp, TRAPS);
   static void init_compiler_threads(int c1_compiler_count, int c2_compiler_count);
   static bool compilation_is_complete  (methodHandle method, int osr_bci, int comp_level);
   static bool compilation_is_prohibited(methodHandle method, int osr_bci, int comp_level);
@@ -351,6 +355,9 @@
     if (is_c1_compile(comp_level)) return _c1_method_queue;
     return NULL;
   }
+  static bool init_compiler_runtime();
+  static void shutdown_compiler_runtime(AbstractCompiler* comp, CompilerThread* thread);
+
  public:
   enum {
     // The entry bci used for non-OSR compilations.
@@ -378,9 +385,7 @@
                                  const char* comment, Thread* thread);
 
   static void compiler_thread_loop();
-
   static uint get_compilation_id() { return _compilation_id; }
-  static bool is_idle();
 
   // Set _should_block.
   // Call this from the VM, with Threads_lock held and a safepoint requested.
@@ -391,8 +396,9 @@
 
   enum {
     // Flags for toggling compiler activity
-    stop_compilation = 0,
-    run_compilation  = 1
+    stop_compilation    = 0,
+    run_compilation     = 1,
+    shutdown_compilaton = 2
   };
 
   static bool should_compile_new_jobs() { return UseCompiler && (_should_compile_new_jobs == run_compilation); }
@@ -401,6 +407,16 @@
     jint old = Atomic::cmpxchg(new_state, &_should_compile_new_jobs, 1-new_state);
     return (old == (1-new_state));
   }
+
+  static void disable_compilation_forever() {
+    UseCompiler               = false;
+    AlwaysCompileLoopMethods  = false;
+    Atomic::xchg(shutdown_compilaton, &_should_compile_new_jobs);
+  }
+
+  static bool is_compilation_disabled_forever() {
+    return _should_compile_new_jobs == shutdown_compilaton;
+  }
   static void handle_full_code_cache();
 
   // Return total compilation ticks
--- a/src/share/vm/opto/c2compiler.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/opto/c2compiler.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -44,9 +44,6 @@
 # include "adfiles/ad_ppc.hpp"
 #endif
 
-
-volatile int C2Compiler::_runtimes = uninitialized;
-
 // register information defined by ADLC
 extern const char register_save_policy[];
 extern const int  register_save_type[];
@@ -57,7 +54,7 @@
 const char* C2Compiler::retry_no_escape_analysis() {
   return "retry without escape analysis";
 }
-void C2Compiler::initialize_runtime() {
+bool C2Compiler::init_c2_runtime() {
 
   // Check assumptions used while running ADLC
   Compile::adlc_verification();
@@ -90,41 +87,31 @@
 
   CompilerThread* thread = CompilerThread::current();
 
-  HandleMark  handle_mark(thread);
-
-  OptoRuntime::generate(thread->env());
-
+  HandleMark handle_mark(thread);
+  return OptoRuntime::generate(thread->env());
 }
 
 
 void C2Compiler::initialize() {
-
-  // This method can only be called once per C2Compiler object
   // The first compiler thread that gets here will initialize the
-  // small amount of global state (and runtime stubs) that c2 needs.
+  // small amount of global state (and runtime stubs) that C2 needs.
 
   // There is a race possible once at startup and then we're fine
 
   // Note that this is being called from a compiler thread not the
   // main startup thread.
-
-  if (_runtimes != initialized) {
-    initialize_runtimes( initialize_runtime, &_runtimes);
+  if (should_perform_init()) {
+    bool successful = C2Compiler::init_c2_runtime();
+    int new_state = (successful) ? initialized : failed;
+    set_state(new_state);
   }
-
-  // Mark this compiler object as ready to roll
-  mark_initialized();
 }
 
-void C2Compiler::compile_method(ciEnv* env,
-                                ciMethod* target,
-                                int entry_bci) {
-  if (!is_initialized()) {
-    initialize();
-  }
+void C2Compiler::compile_method(ciEnv* env, ciMethod* target, int entry_bci) {
+  assert(is_initialized(), "Compiler thread must be initialized");
+
   bool subsume_loads = SubsumeLoads;
-  bool do_escape_analysis = DoEscapeAnalysis &&
-    !env->jvmti_can_access_local_variables();
+  bool do_escape_analysis = DoEscapeAnalysis && !env->jvmti_can_access_local_variables();
   bool eliminate_boxing = EliminateAutoBox;
   while (!env->failing()) {
     // Attempt to compile while subsuming loads into machine instructions.
--- a/src/share/vm/opto/c2compiler.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/opto/c2compiler.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -28,24 +28,17 @@
 #include "compiler/abstractCompiler.hpp"
 
 class C2Compiler : public AbstractCompiler {
-private:
-
-  static void initialize_runtime();
+ private:
+  static bool init_c2_runtime();
 
 public:
   // Name
   const char *name() { return "C2"; }
 
-  static volatile int _runtimes;
-
 #ifdef TIERED
   virtual bool is_c2() { return true; };
 #endif // TIERED
 
-  // Customization
-  bool needs_adapters         () { return true; }
-  bool needs_stubs            () { return true; }
-
   void initialize();
 
   // Compilation entry point for methods
--- a/src/share/vm/opto/runtime.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/opto/runtime.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -138,9 +138,10 @@
 
 
 #define gen(env, var, type_func_gen, c_func, fancy_jump, pass_tls, save_arg_regs, return_pc) \
-  var = generate_stub(env, type_func_gen, CAST_FROM_FN_PTR(address, c_func), #var, fancy_jump, pass_tls, save_arg_regs, return_pc)
+  var = generate_stub(env, type_func_gen, CAST_FROM_FN_PTR(address, c_func), #var, fancy_jump, pass_tls, save_arg_regs, return_pc); \
+  if (var == NULL) { return false; }
 
-void OptoRuntime::generate(ciEnv* env) {
+bool OptoRuntime::generate(ciEnv* env) {
 
   generate_exception_blob();
 
@@ -158,7 +159,7 @@
   gen(env, _multianewarrayN_Java           , multianewarrayN_Type         , multianewarrayN_C               ,    0 , true , false, false);
   gen(env, _g1_wb_pre_Java                 , g1_wb_pre_Type               , SharedRuntime::g1_wb_pre        ,    0 , false, false, false);
   gen(env, _g1_wb_post_Java                , g1_wb_post_Type              , SharedRuntime::g1_wb_post       ,    0 , false, false, false);
-  gen(env, _complete_monitor_locking_Java  , complete_monitor_enter_Type  , SharedRuntime::complete_monitor_locking_C      ,    0 , false, false, false);
+  gen(env, _complete_monitor_locking_Java  , complete_monitor_enter_Type  , SharedRuntime::complete_monitor_locking_C, 0, false, false, false);
   gen(env, _rethrow_Java                   , rethrow_Type                 , rethrow_C                       ,    2 , true , false, true );
 
   gen(env, _slow_arraycopy_Java            , slow_arraycopy_Type          , SharedRuntime::slow_arraycopy_C ,    0 , false, false, false);
@@ -168,7 +169,7 @@
   gen(env, _zap_dead_Java_locals_Java      , zap_dead_locals_Type         , zap_dead_Java_locals_C          ,    0 , false, true , false );
   gen(env, _zap_dead_native_locals_Java    , zap_dead_locals_Type         , zap_dead_native_locals_C        ,    0 , false, true , false );
 # endif
-
+  return true;
 }
 
 #undef gen
--- a/src/share/vm/opto/runtime.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/opto/runtime.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -203,8 +203,10 @@
 
   static bool is_callee_saved_register(MachRegisterNumbers reg);
 
-  // One time only generate runtime code stubs
-  static void generate(ciEnv* env);
+  // One time only generate runtime code stubs. Returns true
+  // when runtime stubs have been generated successfully and
+  // false otherwise.
+  static bool generate(ciEnv* env);
 
   // Returns the name of a stub
   static const char* stub_name(address entry);
--- a/src/share/vm/runtime/thread.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/runtime/thread.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -1454,7 +1454,6 @@
   _interp_only_mode    = 0;
   _special_runtime_exit_condition = _no_async_condition;
   _pending_async_exception = NULL;
-  _is_compiling = false;
   _thread_stat = NULL;
   _thread_stat = new ThreadStatistics();
   _blocked_on_compilation = false;
@@ -1815,7 +1814,8 @@
     // Call Thread.exit(). We try 3 times in case we got another Thread.stop during
     // the execution of the method. If that is not enough, then we don't really care. Thread.stop
     // is deprecated anyhow.
-    { int count = 3;
+    if (!is_Compiler_thread()) {
+      int count = 3;
       while (java_lang_Thread::threadGroup(threadObj()) != NULL && (count-- > 0)) {
         EXCEPTION_MARK;
         JavaValue result(T_VOID);
@@ -1828,7 +1828,6 @@
         CLEAR_PENDING_EXCEPTION;
       }
     }
-
     // notify JVMTI
     if (JvmtiExport::should_post_thread_life()) {
       JvmtiExport::post_thread_end(this);
@@ -3239,6 +3238,7 @@
   _counters = counters;
   _buffer_blob = NULL;
   _scanned_nmethod = NULL;
+  _compiler = NULL;
 
 #ifndef PRODUCT
   _ideal_graph_printer = NULL;
@@ -3255,6 +3255,7 @@
   }
 }
 
+
 // ======= Threads ========
 
 // The Threads class links together all active threads, and provides
@@ -3275,8 +3276,6 @@
 // All JavaThreads
 #define ALL_JAVA_THREADS(X) for (JavaThread* X = _thread_list; X; X = X->next())
 
-void os_stream();
-
 // All JavaThreads + all non-JavaThreads (i.e., every thread in the system)
 void Threads::threads_do(ThreadClosure* tc) {
   assert_locked_or_safepoint(Threads_lock);
--- a/src/share/vm/runtime/thread.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/runtime/thread.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -923,9 +923,6 @@
   volatile address _exception_handler_pc;        // PC for handler of exception
   volatile int     _is_method_handle_return;     // true (== 1) if the current exception PC is a MethodHandle call site.
 
-  // support for compilation
-  bool    _is_compiling;                         // is true if a compilation is active inthis thread (one compilation per thread possible)
-
   // support for JNI critical regions
   jint    _jni_active_critical;                  // count of entries into JNI critical region
 
@@ -1005,10 +1002,6 @@
   // Testers
   virtual bool is_Java_thread() const            { return true;  }
 
-  // compilation
-  void set_is_compiling(bool f)                  { _is_compiling = f; }
-  bool is_compiling() const                      { return _is_compiling; }
-
   // Thread chain operations
   JavaThread* next() const                       { return _next; }
   void set_next(JavaThread* p)                   { _next = p; }
@@ -1816,13 +1809,14 @@
  private:
   CompilerCounters* _counters;
 
-  ciEnv*        _env;
-  CompileLog*   _log;
-  CompileTask*  _task;
-  CompileQueue* _queue;
-  BufferBlob*   _buffer_blob;
+  ciEnv*            _env;
+  CompileLog*       _log;
+  CompileTask*      _task;
+  CompileQueue*     _queue;
+  BufferBlob*       _buffer_blob;
 
-  nmethod*      _scanned_nmethod;  // nmethod being scanned by the sweeper
+  nmethod*          _scanned_nmethod;  // nmethod being scanned by the sweeper
+  AbstractCompiler* _compiler;
 
  public:
 
@@ -1834,14 +1828,17 @@
   // Hide this compiler thread from external view.
   bool is_hidden_from_external_view() const      { return true; }
 
-  CompileQueue* queue()                          { return _queue; }
-  CompilerCounters* counters()                   { return _counters; }
+  void set_compiler(AbstractCompiler* c)         { _compiler = c; }
+  AbstractCompiler* compiler() const             { return _compiler; }
+
+  CompileQueue* queue()        const             { return _queue; }
+  CompilerCounters* counters() const             { return _counters; }
 
   // Get/set the thread's compilation environment.
   ciEnv*        env()                            { return _env; }
   void          set_env(ciEnv* env)              { _env = env; }
 
-  BufferBlob*   get_buffer_blob()                { return _buffer_blob; }
+  BufferBlob*   get_buffer_blob() const          { return _buffer_blob; }
   void          set_buffer_blob(BufferBlob* b)   { _buffer_blob = b; };
 
   // Get/set the thread's logging information
--- a/src/share/vm/runtime/vmStructs.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/runtime/vmStructs.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -910,7 +910,6 @@
   volatile_nonstatic_field(JavaThread,         _exception_oop,                                oop)                                   \
   volatile_nonstatic_field(JavaThread,         _exception_pc,                                 address)                               \
   volatile_nonstatic_field(JavaThread,         _is_method_handle_return,                      int)                                   \
-  nonstatic_field(JavaThread,                  _is_compiling,                                 bool)                                  \
   nonstatic_field(JavaThread,                  _special_runtime_exit_condition,               JavaThread::AsyncRequests)             \
   nonstatic_field(JavaThread,                  _saved_exception_pc,                           address)                               \
    volatile_nonstatic_field(JavaThread,        _thread_state,                                 JavaThreadState)                       \
--- a/src/share/vm/shark/sharkCompiler.cpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/shark/sharkCompiler.cpp	Thu Oct 10 15:44:12 2013 +0200
@@ -133,11 +133,10 @@
     exit(1);
   }
 
-  execution_engine()->addModule(
-    _native_context->module());
+  execution_engine()->addModule(_native_context->module());
 
   // All done
-  mark_initialized();
+  set_state(initialized);
 }
 
 void SharkCompiler::initialize() {
--- a/src/share/vm/shark/sharkCompiler.hpp	Fri Oct 11 10:14:02 2013 -0700
+++ b/src/share/vm/shark/sharkCompiler.hpp	Thu Oct 10 15:44:12 2013 +0200
@@ -50,10 +50,6 @@
     return ! (method->is_method_handle_intrinsic() || method->is_compiled_lambda_form());
   }
 
-  // Customization
-  bool needs_adapters()  { return false; }
-  bool needs_stubs()     { return false; }
-
   // Initialization
   void initialize();
 
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/test/compiler/startup/SmallCodeCacheStartup.java	Thu Oct 10 15:44:12 2013 +0200
@@ -0,0 +1,43 @@
+/*
+ * Copyright (c) 2013, 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.
+ *
+ * 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.
+ */
+
+/*
+ * @test
+ * @bug 8023014
+ * @summary Test ensures that there is no crash when compiler initialization fails
+ * @library /testlibrary
+ *
+ */
+import com.oracle.java.testlibrary.*;
+
+public class SmallCodeCacheStartup {
+  public static void main(String[] args) throws Exception {
+    ProcessBuilder pb;
+    OutputAnalyzer out;
+
+    pb = ProcessTools.createJavaProcessBuilder("-XX:ReservedCodeCacheSize=3m", "-XX:CICompilerCount=64", "-version");
+    out = new OutputAnalyzer(pb.start());
+    out.shouldContain("no space to run compiler");
+    out.shouldHaveExitValue(0);
+  }
+}