From fe1158fb6747a2804b34278e0a48ad9754243e06 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Fri, 1 Nov 2024 23:10:31 +0100 Subject: [PATCH 01/34] Decouple workers. --- frankenphp.c | 84 ++++++++++++++++++++++++++++----------------- frankenphp.go | 36 ++++++++++++++++++-- frankenphp.h | 4 ++- php_thread.go | 15 +++++++- worker.go | 94 +++++++++++++++++++++++++++++++++++++++++++-------- 5 files changed, 183 insertions(+), 50 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 54c149763b..ce71f7becf 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -838,10 +838,10 @@ static void set_thread_name(char *thread_name) { #endif } -static void *php_thread(void *arg) { - char thread_name[16] = {0}; - snprintf(thread_name, 16, "php-%" PRIxPTR, (uintptr_t)arg); +static void init_php_thread(void *arg) { thread_index = (uintptr_t)arg; + char thread_name[16] = {0}; + snprintf(thread_name, 16, "php-%" PRIxPTR, thread_index); set_thread_name(thread_name); #ifdef ZTS @@ -853,14 +853,41 @@ static void *php_thread(void *arg) { #endif local_ctx = malloc(sizeof(frankenphp_server_context)); +} +static void shutdown_php_thread(void) { + //free(local_ctx); + //local_ctx = NULL; +#ifdef ZTS + ts_free_thread(); +#endif +} +static void *php_thread(void *arg) { + init_php_thread(arg); + + // handle requests until the channel is closed while (go_handle_request(thread_index)) { } -#ifdef ZTS - ts_free_thread(); -#endif + shutdown_php_thread(); + return NULL; +} + +static void *php_worker_thread(void *arg) { + init_php_thread(arg); + + // run the loop that executes the worker script + while (true) { + char *script_name = go_before_worker_script(thread_index); + if (script_name == NULL) { + break; + } + frankenphp_execute_script(script_name); + go_after_worker_script(thread_index); + } + shutdown_php_thread(); + go_shutdown_woker_thread(thread_index); return NULL; } @@ -912,28 +939,7 @@ static void *php_main(void *arg) { frankenphp_sapi_module.startup(&frankenphp_sapi_module); - pthread_t *threads = malloc(num_threads * sizeof(pthread_t)); - if (threads == NULL) { - perror("malloc failed"); - exit(EXIT_FAILURE); - } - - for (uintptr_t i = 0; i < num_threads; i++) { - if (pthread_create(&(*(threads + i)), NULL, &php_thread, (void *)i) != 0) { - perror("failed to create PHP thread"); - free(threads); - exit(EXIT_FAILURE); - } - } - - for (int i = 0; i < num_threads; i++) { - if (pthread_join((*(threads + i)), NULL) != 0) { - perror("failed to join PHP thread"); - free(threads); - exit(EXIT_FAILURE); - } - } - free(threads); + go_listen_for_shutdown(); /* channel closed, shutdown gracefully */ frankenphp_sapi_module.shutdown(&frankenphp_sapi_module); @@ -955,19 +961,35 @@ static void *php_main(void *arg) { return NULL; } -int frankenphp_init(int num_threads) { +int frankenphp_new_main_thread(int num_threads) { pthread_t thread; if (pthread_create(&thread, NULL, &php_main, (void *)(intptr_t)num_threads) != 0) { go_shutdown(); - return -1; } - return pthread_detach(thread); } +int frankenphp_new_worker_thread(uintptr_t thread_index){ + pthread_t thread; + if (pthread_create(&thread, NULL, &php_worker_thread, (void *)thread_index) != 0){ + return 1; + } + pthread_detach(thread); + return 0; +} + +int frankenphp_new_php_thread(uintptr_t thread_index){ + pthread_t thread; + if (pthread_create(&thread, NULL, &php_thread, (void *)thread_index) != 0){ + return 1; + } + pthread_detach(thread); + return 0; +} + int frankenphp_request_startup() { if (php_request_startup() == SUCCESS) { return SUCCESS; diff --git a/frankenphp.go b/frankenphp.go index 2882a1c177..5bbbaeae07 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -65,6 +65,7 @@ var ( requestChan chan *http.Request done chan struct{} + mainThreadWG sync.WaitGroup shutdownWG sync.WaitGroup loggerMu sync.RWMutex @@ -336,8 +337,13 @@ func Init(options ...Option) error { requestChan = make(chan *http.Request) initPHPThreads(opt.numThreads) - if C.frankenphp_init(C.int(opt.numThreads)) != 0 { - return MainThreadCreationError + startMainThread(opt.numThreads) + + // TODO: calc num threads + for i := 0; i < 1; i++ { + if err := startNewThread(); err != nil { + return err + } } if err := initWorkers(opt.workers); err != nil { @@ -386,6 +392,24 @@ func drainThreads() { phpThreads = nil } +func startMainThread(numThreads int) error { + mainThreadWG.Add(1) + if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { + return MainThreadCreationError + } + mainThreadWG.Wait() + return nil +} + +func startNewThread() error { + thread := getInactiveThread() + thread.isActive = true + if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("error creating thread %d", thread.threadIndex) + } + return nil +} + func getLogger() *zap.Logger { loggerMu.RLock() defer loggerMu.RUnlock() @@ -505,6 +529,14 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error return nil } +//export go_listen_for_shutdown +func go_listen_for_shutdown(){ + mainThreadWG.Done() + select{ + case <-done: + } +} + //export go_putenv func go_putenv(str *C.char, length C.int) C.bool { // Create a byte slice from C string with a specified length diff --git a/frankenphp.h b/frankenphp.h index a0c54936dd..7470ba00e4 100644 --- a/frankenphp.h +++ b/frankenphp.h @@ -40,7 +40,9 @@ typedef struct frankenphp_config { } frankenphp_config; frankenphp_config frankenphp_get_config(); -int frankenphp_init(int num_threads); +int frankenphp_new_main_thread(int num_threads); +int frankenphp_new_php_thread(uintptr_t thread_index); +int frankenphp_new_worker_thread(uintptr_t thread_index); int frankenphp_update_server_context( bool create, bool has_main_request, bool has_active_request, diff --git a/php_thread.go b/php_thread.go index 5b9c299700..608040b935 100644 --- a/php_thread.go +++ b/php_thread.go @@ -15,15 +15,28 @@ type phpThread struct { mainRequest *http.Request workerRequest *http.Request worker *worker + isActive bool + isReady bool + threadIndex int } func initPHPThreads(numThreads int) { phpThreads = make([]*phpThread, 0, numThreads) for i := 0; i < numThreads; i++ { - phpThreads = append(phpThreads, &phpThread{}) + phpThreads = append(phpThreads, &phpThread{threadIndex: i}) } } +func getInactiveThread() *phpThread { + for _, thread := range phpThreads { + if !thread.isActive { + return thread + } + } + + return nil +} + func (thread phpThread) getActiveRequest() *http.Request { if thread.workerRequest != nil { return thread.workerRequest diff --git a/worker.go b/worker.go index 38e4b60a4a..0dd71d1fb3 100644 --- a/worker.go +++ b/worker.go @@ -47,9 +47,10 @@ func initWorkers(opt []workerOpt) error { if err != nil { return err } - workersReadyWG.Add(worker.num) for i := 0; i < worker.num; i++ { - go worker.startNewWorkerThread() + if err := worker.startNewThread(nil); err != nil { + return err + } } } @@ -82,6 +83,19 @@ func newWorker(o workerOpt) (*worker, error) { return w, nil } +func (worker *worker) startNewThread(r *http.Request) error { + workersReadyWG.Add(1) + workerShutdownWG.Add(1) + thread := getInactiveThread() + thread.worker = worker + thread.isActive = true + if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("failed to create worker thread") + } + + return nil +} + func (worker *worker) startNewWorkerThread() { workerShutdownWG.Add(1) defer workerShutdownWG.Done() @@ -232,26 +246,76 @@ func restartWorkers(workerOpts []workerOpt) { } func assignThreadToWorker(thread *phpThread) { - fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) - metrics.ReadyWorker(fc.scriptFilename) - worker, ok := workers[fc.scriptFilename] - if !ok { - panic("worker not found for script: " + fc.scriptFilename) - } - thread.worker = worker - if !workersAreReady.Load() { - workersReadyWG.Done() - } + metrics.ReadyWorker(thread.worker.fileName) + thread.isReady = true + workersReadyWG.Done() // TODO: we can also store all threads assigned to the worker if needed } +//export go_before_worker_script +func go_before_worker_script(threadIndex C.uintptr_t) *C.char { + thread := phpThreads[threadIndex] + worker := thread.worker + + // if we are done, exit the loop that restarts the worker script + if workersAreDone.Load() { + return nil + } + metrics.StartWorker(worker.fileName) + + // Create main dummy request + r, err := http.NewRequest(http.MethodGet, filepath.Base(worker.fileName), nil) + if err != nil { + panic(err) + } + + r, err = NewRequestWithContext( + r, + WithRequestDocumentRoot(filepath.Dir(worker.fileName), false), + WithRequestPreparedEnv(worker.env), + ) + if err != nil { + panic(err) + } + thread.mainRequest = r + if c := logger.Check(zapcore.DebugLevel, "starting"); c != nil { + c.Write(zap.String("worker", worker.fileName), zap.Int("num", worker.num)) + } + + if err := updateServerContext(r, true, false); err != nil { + panic(err) + } + return C.CString(worker.fileName) +} + +//export go_after_worker_script +func go_after_worker_script(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) + + // on exit status 0 we just run the worker script again + if fc.exitStatus == 0 { + // TODO: make the max restart configurable + if c := logger.Check(zapcore.InfoLevel, "restarting"); c != nil { + c.Write(zap.String("worker", thread.worker.fileName)) + } + metrics.StopWorker(thread.worker.fileName, StopReasonRestart) + } +} + +//export go_shutdown_woker_thread +func go_shutdown_woker_thread(threadIndex C.uintptr_t) { + workerShutdownWG.Done() +} + //export go_frankenphp_worker_handle_request_start func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { thread := phpThreads[threadIndex] - // we assign a worker to the thread if it doesn't have one already - if thread.worker == nil { - assignThreadToWorker(thread) + if !thread.isReady { + thread.isReady = true + workersReadyWG.Done() + metrics.ReadyWorker(thread.worker.fileName) } if c := logger.Check(zapcore.DebugLevel, "waiting for request"); c != nil { From ad34140027c311b6e11d4e9755a691be98c7ace6 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 00:43:59 +0100 Subject: [PATCH 02/34] Moves code to separate file. --- frankenphp.c | 18 ++++---- frankenphp.go | 65 +++++++-------------------- php_thread.go | 19 -------- php_thread_test.go | 16 +------ php_threads.go | 108 +++++++++++++++++++++++++++++++++++++++++++++ worker.go | 36 ++++++--------- 6 files changed, 148 insertions(+), 114 deletions(-) create mode 100644 php_threads.go diff --git a/frankenphp.c b/frankenphp.c index ce71f7becf..93c283e3d8 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -855,8 +855,8 @@ static void init_php_thread(void *arg) { local_ctx = malloc(sizeof(frankenphp_server_context)); } static void shutdown_php_thread(void) { - //free(local_ctx); - //local_ctx = NULL; + free(local_ctx); + local_ctx = NULL; #ifdef ZTS ts_free_thread(); #endif @@ -870,6 +870,7 @@ static void *php_thread(void *arg) { } shutdown_php_thread(); + go_shutdown_php_thread(thread_index); return NULL; } @@ -882,12 +883,12 @@ static void *php_worker_thread(void *arg) { if (script_name == NULL) { break; } - frankenphp_execute_script(script_name); - go_after_worker_script(thread_index); + int exit_status = frankenphp_execute_script(script_name); + go_after_worker_script(thread_index, exit_status); } shutdown_php_thread(); - go_shutdown_woker_thread(thread_index); + go_shutdown_worker_thread(thread_index); return NULL; } @@ -939,7 +940,7 @@ static void *php_main(void *arg) { frankenphp_sapi_module.startup(&frankenphp_sapi_module); - go_listen_for_shutdown(); + go_main_thread_is_ready(); /* channel closed, shutdown gracefully */ frankenphp_sapi_module.shutdown(&frankenphp_sapi_module); @@ -955,9 +956,7 @@ static void *php_main(void *arg) { frankenphp_sapi_module.ini_entries = NULL; } #endif - - go_shutdown(); - + go_shutdown_main_thread(); return NULL; } @@ -966,7 +965,6 @@ int frankenphp_new_main_thread(int num_threads) { if (pthread_create(&thread, NULL, &php_main, (void *)(intptr_t)num_threads) != 0) { - go_shutdown(); return -1; } return pthread_detach(thread); diff --git a/frankenphp.go b/frankenphp.go index 5bbbaeae07..f53870cbbf 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -65,8 +65,6 @@ var ( requestChan chan *http.Request done chan struct{} - mainThreadWG sync.WaitGroup - shutdownWG sync.WaitGroup loggerMu sync.RWMutex logger *zap.Logger @@ -332,16 +330,19 @@ func Init(options ...Option) error { logger.Warn(`ZTS is not enabled, only 1 thread will be available, recompile PHP using the "--enable-zts" configuration option or performance will be degraded`) } - shutdownWG.Add(1) done = make(chan struct{}) requestChan = make(chan *http.Request) - initPHPThreads(opt.numThreads) + if err:= initPHPThreads(opt.numThreads); err != nil { + return err + } - startMainThread(opt.numThreads) + totalWorkers := 0 + for _, w := range opt.workers { + totalWorkers += w.num + } - // TODO: calc num threads - for i := 0; i < 1; i++ { - if err := startNewThread(); err != nil { + for i := 0; i < opt.numThreads - totalWorkers; i++ { + if err := startNewPHPThread(); err != nil { return err } } @@ -349,6 +350,7 @@ func Init(options ...Option) error { if err := initWorkers(opt.workers); err != nil { return err } + readyWG.Wait() if err := restartWorkersOnFileChanges(opt.workers); err != nil { return err @@ -369,7 +371,7 @@ func Init(options ...Option) error { // Shutdown stops the workers and the PHP runtime. func Shutdown() { drainWorkers() - drainThreads() + drainPHPThreads() metrics.Shutdown() requestChan = nil @@ -381,35 +383,6 @@ func Shutdown() { logger.Debug("FrankenPHP shut down") } -//export go_shutdown -func go_shutdown() { - shutdownWG.Done() -} - -func drainThreads() { - close(done) - shutdownWG.Wait() - phpThreads = nil -} - -func startMainThread(numThreads int) error { - mainThreadWG.Add(1) - if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { - return MainThreadCreationError - } - mainThreadWG.Wait() - return nil -} - -func startNewThread() error { - thread := getInactiveThread() - thread.isActive = true - if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("error creating thread %d", thread.threadIndex) - } - return nil -} - func getLogger() *zap.Logger { loggerMu.RLock() defer loggerMu.RUnlock() @@ -486,9 +459,6 @@ func updateServerContext(request *http.Request, create bool, isWorkerRequest boo // ServeHTTP executes a PHP script according to the given context. func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error { - shutdownWG.Add(1) - defer shutdownWG.Done() - fc, ok := FromContext(request.Context()) if !ok { return InvalidRequestError @@ -529,14 +499,6 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error return nil } -//export go_listen_for_shutdown -func go_listen_for_shutdown(){ - mainThreadWG.Done() - select{ - case <-done: - } -} - //export go_putenv func go_putenv(str *C.char, length C.int) C.bool { // Create a byte slice from C string with a specified length @@ -609,6 +571,11 @@ func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string //export go_handle_request func go_handle_request(threadIndex C.uintptr_t) bool { + thread := phpThreads[threadIndex] + if !thread.isReady { + thread.isReady = true + readyWG.Done() + } select { case <-done: return false diff --git a/php_thread.go b/php_thread.go index 608040b935..5611a1d04a 100644 --- a/php_thread.go +++ b/php_thread.go @@ -7,8 +7,6 @@ import ( "runtime" ) -var phpThreads []*phpThread - type phpThread struct { runtime.Pinner @@ -20,23 +18,6 @@ type phpThread struct { threadIndex int } -func initPHPThreads(numThreads int) { - phpThreads = make([]*phpThread, 0, numThreads) - for i := 0; i < numThreads; i++ { - phpThreads = append(phpThreads, &phpThread{threadIndex: i}) - } -} - -func getInactiveThread() *phpThread { - for _, thread := range phpThreads { - if !thread.isActive { - return thread - } - } - - return nil -} - func (thread phpThread) getActiveRequest() *http.Request { if thread.workerRequest != nil { return thread.workerRequest diff --git a/php_thread_test.go b/php_thread_test.go index 63afe4d89c..eba873d5be 100644 --- a/php_thread_test.go +++ b/php_thread_test.go @@ -7,20 +7,9 @@ import ( "github.com/stretchr/testify/assert" ) -func TestInitializeTwoPhpThreadsWithoutRequests(t *testing.T) { - initPHPThreads(2) - - assert.Len(t, phpThreads, 2) - assert.NotNil(t, phpThreads[0]) - assert.NotNil(t, phpThreads[1]) - assert.Nil(t, phpThreads[0].mainRequest) - assert.Nil(t, phpThreads[0].workerRequest) -} - func TestMainRequestIsActiveRequest(t *testing.T) { mainRequest := &http.Request{} - initPHPThreads(1) - thread := phpThreads[0] + thread := phpThread{} thread.mainRequest = mainRequest @@ -30,8 +19,7 @@ func TestMainRequestIsActiveRequest(t *testing.T) { func TestWorkerRequestIsActiveRequest(t *testing.T) { mainRequest := &http.Request{} workerRequest := &http.Request{} - initPHPThreads(1) - thread := phpThreads[0] + thread := phpThread{} thread.mainRequest = mainRequest thread.workerRequest = workerRequest diff --git a/php_threads.go b/php_threads.go new file mode 100644 index 0000000000..417bfa75e8 --- /dev/null +++ b/php_threads.go @@ -0,0 +1,108 @@ +package frankenphp + +// #include +// #include "frankenphp.h" +import "C" +import ( + "fmt" + "sync" +) + +var ( + phpThreads []*phpThread + mainThreadWG sync.WaitGroup + terminationWG sync.WaitGroup + mainThreadShutdownWG sync.WaitGroup + readyWG sync.WaitGroup + shutdownWG sync.WaitGroup +) + +// reserve a fixed number of PHP threads on the go side +func initPHPThreads(numThreads int) error { + phpThreads = make([]*phpThread, numThreads) + for i := 0; i < numThreads; i++ { + phpThreads[i] = &phpThread{threadIndex: i} + } + return startMainThread(numThreads) +} + +func drainPHPThreads() { + close(done) + shutdownWG.Wait() + phpThreads = nil + mainThreadShutdownWG.Done() + terminationWG.Wait() +} + +func startMainThread(numThreads int) error { + mainThreadWG.Add(1) + mainThreadShutdownWG.Add(1) + terminationWG.Add(1) + if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { + return MainThreadCreationError + } + mainThreadWG.Wait() + return nil +} + +func startNewPHPThread() error { + readyWG.Add(1) + shutdownWG.Add(1) + thread := getInactiveThread() + thread.isActive = true + if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("error creating thread %d", thread.threadIndex) + } + return nil +} + +func startNewWorkerThread(worker *worker) error { + workersReadyWG.Add(1) + workerShutdownWG.Add(1) + thread := getInactiveThread() + thread.worker = worker + thread.isActive = true + if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("failed to create worker thread") + } + + return nil +} + +func getInactiveThread() *phpThread { + for _, thread := range phpThreads { + if !thread.isActive { + return thread + } + } + + return nil +} + +//export go_main_thread_is_ready +func go_main_thread_is_ready(){ + mainThreadWG.Done() + mainThreadShutdownWG.Wait() +} + +//export go_shutdown_main_thread +func go_shutdown_main_thread(){ + terminationWG.Done() +} + +//export go_shutdown_php_thread +func go_shutdown_php_thread(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + thread.Unpin() + thread.isActive = false + shutdownWG.Done() +} + +//export go_shutdown_worker_thread +func go_shutdown_worker_thread(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + thread.Unpin() + thread.isActive = false + thread.worker = nil + workerShutdownWG.Done() +} \ No newline at end of file diff --git a/worker.go b/worker.go index 0dd71d1fb3..eaa8e2a6db 100644 --- a/worker.go +++ b/worker.go @@ -48,7 +48,7 @@ func initWorkers(opt []workerOpt) error { return err } for i := 0; i < worker.num; i++ { - if err := worker.startNewThread(nil); err != nil { + if err := startNewWorkerThread(worker); err != nil { return err } } @@ -83,20 +83,7 @@ func newWorker(o workerOpt) (*worker, error) { return w, nil } -func (worker *worker) startNewThread(r *http.Request) error { - workersReadyWG.Add(1) - workerShutdownWG.Add(1) - thread := getInactiveThread() - thread.worker = worker - thread.isActive = true - if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("failed to create worker thread") - } - - return nil -} - -func (worker *worker) startNewWorkerThread() { +func (worker *worker) asdasd() { workerShutdownWG.Add(1) defer workerShutdownWG.Done() @@ -289,10 +276,14 @@ func go_before_worker_script(threadIndex C.uintptr_t) *C.char { } //export go_after_worker_script -func go_after_worker_script(threadIndex C.uintptr_t) { +func go_after_worker_script(threadIndex C.uintptr_t, exitStatus C.int) { thread := phpThreads[threadIndex] fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) + fc.exitStatus = exitStatus + if fc.exitStatus < 0 { + panic(ScriptExecutionError) + } // on exit status 0 we just run the worker script again if fc.exitStatus == 0 { // TODO: make the max restart configurable @@ -300,12 +291,14 @@ func go_after_worker_script(threadIndex C.uintptr_t) { c.Write(zap.String("worker", thread.worker.fileName)) } metrics.StopWorker(thread.worker.fileName, StopReasonRestart) + return + } else { + time.Sleep(1 * time.Millisecond) + logger.Error("worker script exited with non-zero status") } -} - -//export go_shutdown_woker_thread -func go_shutdown_woker_thread(threadIndex C.uintptr_t) { - workerShutdownWG.Done() + maybeCloseContext(fc) + thread.mainRequest = nil + thread.Unpin() } //export go_frankenphp_worker_handle_request_start @@ -328,7 +321,6 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { if c := logger.Check(zapcore.DebugLevel, "shutting down"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) } - thread.worker = nil executePHPFunction("opcache_reset") return C.bool(false) From 89b211d678e328a7de4995406119a781be90e80a Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 12:53:31 +0100 Subject: [PATCH 03/34] Cleans up the exponential backoff. --- exponential_backoff.go | 60 +++++++++++ frankenphp.go | 13 ++- php_thread.go | 21 +++- php_threads.go | 50 ++++----- worker.go | 232 +++++++++++------------------------------ 5 files changed, 170 insertions(+), 206 deletions(-) create mode 100644 exponential_backoff.go diff --git a/exponential_backoff.go b/exponential_backoff.go new file mode 100644 index 0000000000..359e2bd4fa --- /dev/null +++ b/exponential_backoff.go @@ -0,0 +1,60 @@ +package frankenphp + +import ( + "sync" + "time" +) + +const maxBackoff = 1 * time.Second +const minBackoff = 100 * time.Millisecond +const maxConsecutiveFailures = 6 + +type exponentialBackoff struct { + backoff time.Duration + failureCount int + mu sync.RWMutex + upFunc sync.Once +} + +func newExponentialBackoff() *exponentialBackoff { + return &exponentialBackoff{backoff: minBackoff} +} + +func (e *exponentialBackoff) reset() { + e.mu.Lock() + e.upFunc = sync.Once{} + wait := e.backoff * 2 + e.mu.Unlock() + go func() { + time.Sleep(wait) + e.mu.Lock() + defer e.mu.Unlock() + e.upFunc.Do(func() { + // if we come back to a stable state, reset the failure count + if e.backoff == minBackoff { + e.failureCount = 0 + } + + // earn back the backoff over time + if e.failureCount > 0 { + e.backoff = max(e.backoff/2, minBackoff) + } + }) + }() +} + +func (e *exponentialBackoff) trigger(onMaxFailures func(failureCount int)) { + e.mu.RLock() + e.upFunc.Do(func() { + if e.failureCount >= maxConsecutiveFailures { + onMaxFailures(e.failureCount) + } + e.failureCount += 1 + }) + wait := e.backoff + e.mu.RUnlock() + time.Sleep(wait) + e.mu.Lock() + e.backoff = min(e.backoff*2, maxBackoff) + e.mu.Unlock() +} diff --git a/frankenphp.go b/frankenphp.go index f53870cbbf..df8d99af64 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -332,7 +332,7 @@ func Init(options ...Option) error { done = make(chan struct{}) requestChan = make(chan *http.Request) - if err:= initPHPThreads(opt.numThreads); err != nil { + if err := initPHPThreads(opt.numThreads); err != nil { return err } @@ -341,7 +341,7 @@ func Init(options ...Option) error { totalWorkers += w.num } - for i := 0; i < opt.numThreads - totalWorkers; i++ { + for i := 0; i < opt.numThreads-totalWorkers; i++ { if err := startNewPHPThread(); err != nil { return err } @@ -350,7 +350,9 @@ func Init(options ...Option) error { if err := initWorkers(opt.workers); err != nil { return err } - readyWG.Wait() + + // wait for all regular and worker threads to be ready for requests + threadsReadyWG.Wait() if err := restartWorkersOnFileChanges(opt.workers); err != nil { return err @@ -572,10 +574,7 @@ func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string //export go_handle_request func go_handle_request(threadIndex C.uintptr_t) bool { thread := phpThreads[threadIndex] - if !thread.isReady { - thread.isReady = true - readyWG.Done() - } + thread.setReadyForRequests() select { case <-done: return false diff --git a/php_thread.go b/php_thread.go index 5611a1d04a..811c7a6777 100644 --- a/php_thread.go +++ b/php_thread.go @@ -13,9 +13,10 @@ type phpThread struct { mainRequest *http.Request workerRequest *http.Request worker *worker - isActive bool - isReady bool - threadIndex int + isActive bool // whether the thread is currently running + isReady bool // whether the thread is ready to accept requests + threadIndex int // the index of the thread in the phpThreads slice + backoff *exponentialBackoff // backoff for worker failures } func (thread phpThread) getActiveRequest() *http.Request { @@ -25,3 +26,17 @@ func (thread phpThread) getActiveRequest() *http.Request { return thread.mainRequest } + +func (thread *phpThread) setReadyForRequests() { + if thread.isReady { + return + } + + thread.isReady = true + threadsReadyWG.Done() + if thread.worker != nil { + metrics.ReadyWorker(thread.worker.fileName) + } +} + + diff --git a/php_threads.go b/php_threads.go index 417bfa75e8..55e19f53ff 100644 --- a/php_threads.go +++ b/php_threads.go @@ -9,12 +9,11 @@ import ( ) var ( - phpThreads []*phpThread - mainThreadWG sync.WaitGroup - terminationWG sync.WaitGroup + phpThreads []*phpThread + terminationWG sync.WaitGroup mainThreadShutdownWG sync.WaitGroup - readyWG sync.WaitGroup - shutdownWG sync.WaitGroup + threadsReadyWG sync.WaitGroup + shutdownWG sync.WaitGroup ) // reserve a fixed number of PHP threads on the go side @@ -35,18 +34,18 @@ func drainPHPThreads() { } func startMainThread(numThreads int) error { - mainThreadWG.Add(1) + threadsReadyWG.Add(1) mainThreadShutdownWG.Add(1) terminationWG.Add(1) - if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { - return MainThreadCreationError - } - mainThreadWG.Wait() - return nil + if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { + return MainThreadCreationError + } + threadsReadyWG.Wait() + return nil } func startNewPHPThread() error { - readyWG.Add(1) + threadsReadyWG.Add(1) shutdownWG.Add(1) thread := getInactiveThread() thread.isActive = true @@ -57,14 +56,15 @@ func startNewPHPThread() error { } func startNewWorkerThread(worker *worker) error { - workersReadyWG.Add(1) - workerShutdownWG.Add(1) + threadsReadyWG.Add(1) + workerShutdownWG.Add(1) thread := getInactiveThread() - thread.worker = worker - thread.isActive = true - if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("failed to create worker thread") - } + thread.worker = worker + thread.backoff = newExponentialBackoff() + thread.isActive = true + if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("failed to create worker thread") + } return nil } @@ -80,13 +80,13 @@ func getInactiveThread() *phpThread { } //export go_main_thread_is_ready -func go_main_thread_is_ready(){ - mainThreadWG.Done() +func go_main_thread_is_ready() { + threadsReadyWG.Done() mainThreadShutdownWG.Wait() } //export go_shutdown_main_thread -func go_shutdown_main_thread(){ +func go_shutdown_main_thread() { terminationWG.Done() } @@ -95,6 +95,7 @@ func go_shutdown_php_thread(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] thread.Unpin() thread.isActive = false + thread.isReady = false shutdownWG.Done() } @@ -103,6 +104,7 @@ func go_shutdown_worker_thread(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] thread.Unpin() thread.isActive = false + thread.isReady = false thread.worker = nil - workerShutdownWG.Done() -} \ No newline at end of file + workerShutdownWG.Done() +} diff --git a/worker.go b/worker.go index eaa8e2a6db..96acc82c72 100644 --- a/worker.go +++ b/worker.go @@ -9,7 +9,6 @@ import ( "path/filepath" "sync" "sync/atomic" - "time" "github.com/dunglas/frankenphp/internal/watcher" "go.uber.org/zap" @@ -23,15 +22,9 @@ type worker struct { requestChan chan *http.Request } -const maxWorkerErrorBackoff = 1 * time.Second -const minWorkerErrorBackoff = 100 * time.Millisecond -const maxWorkerConsecutiveFailures = 6 - var ( watcherIsEnabled bool - workersReadyWG sync.WaitGroup workerShutdownWG sync.WaitGroup - workersAreReady atomic.Bool workersAreDone atomic.Bool workersDone chan interface{} workers = make(map[string]*worker) @@ -39,7 +32,6 @@ var ( func initWorkers(opt []workerOpt) error { workersDone = make(chan interface{}) - workersAreReady.Store(false) workersAreDone.Store(false) for _, o := range opt { @@ -54,9 +46,6 @@ func initWorkers(opt []workerOpt) error { } } - workersReadyWG.Wait() - workersAreReady.Store(true) - return nil } @@ -83,113 +72,6 @@ func newWorker(o workerOpt) (*worker, error) { return w, nil } -func (worker *worker) asdasd() { - workerShutdownWG.Add(1) - defer workerShutdownWG.Done() - - backoff := minWorkerErrorBackoff - failureCount := 0 - backingOffLock := sync.RWMutex{} - - for { - - // if the worker can stay up longer than backoff*2, it is probably an application error - upFunc := sync.Once{} - go func() { - backingOffLock.RLock() - wait := backoff * 2 - backingOffLock.RUnlock() - time.Sleep(wait) - upFunc.Do(func() { - backingOffLock.Lock() - defer backingOffLock.Unlock() - // if we come back to a stable state, reset the failure count - if backoff == minWorkerErrorBackoff { - failureCount = 0 - } - - // earn back the backoff over time - if failureCount > 0 { - backoff = max(backoff/2, 100*time.Millisecond) - } - }) - }() - - metrics.StartWorker(worker.fileName) - - // Create main dummy request - r, err := http.NewRequest(http.MethodGet, filepath.Base(worker.fileName), nil) - if err != nil { - panic(err) - } - - r, err = NewRequestWithContext( - r, - WithRequestDocumentRoot(filepath.Dir(worker.fileName), false), - WithRequestPreparedEnv(worker.env), - ) - if err != nil { - panic(err) - } - - if c := logger.Check(zapcore.DebugLevel, "starting"); c != nil { - c.Write(zap.String("worker", worker.fileName), zap.Int("num", worker.num)) - } - - if err := ServeHTTP(nil, r); err != nil { - panic(err) - } - - fc := r.Context().Value(contextKey).(*FrankenPHPContext) - - // if we are done, exit the loop that restarts the worker script - if workersAreDone.Load() { - break - } - - // on exit status 0 we just run the worker script again - if fc.exitStatus == 0 { - // TODO: make the max restart configurable - if c := logger.Check(zapcore.InfoLevel, "restarting"); c != nil { - c.Write(zap.String("worker", worker.fileName)) - } - metrics.StopWorker(worker.fileName, StopReasonRestart) - continue - } - - // on exit status 1 we log the error and apply an exponential backoff when restarting - upFunc.Do(func() { - backingOffLock.Lock() - defer backingOffLock.Unlock() - // if we end up here, the worker has not been up for backoff*2 - // this is probably due to a syntax error or another fatal error - if failureCount >= maxWorkerConsecutiveFailures { - if !watcherIsEnabled { - panic(fmt.Errorf("workers %q: too many consecutive failures", worker.fileName)) - } - logger.Warn("many consecutive worker failures", zap.String("worker", worker.fileName), zap.Int("failures", failureCount)) - } - failureCount += 1 - }) - backingOffLock.RLock() - wait := backoff - backingOffLock.RUnlock() - time.Sleep(wait) - backingOffLock.Lock() - backoff *= 2 - backoff = min(backoff, maxWorkerErrorBackoff) - backingOffLock.Unlock() - metrics.StopWorker(worker.fileName, StopReasonCrash) - } - - metrics.StopWorker(worker.fileName, StopReasonShutdown) - - // TODO: check if the termination is expected - if c := logger.Check(zapcore.DebugLevel, "terminated"); c != nil { - c.Write(zap.String("worker", worker.fileName)) - } -} - func stopWorkers() { workersAreDone.Store(true) close(workersDone) @@ -232,13 +114,6 @@ func restartWorkers(workerOpts []workerOpt) { logger.Info("workers restarted successfully") } -func assignThreadToWorker(thread *phpThread) { - metrics.ReadyWorker(thread.worker.fileName) - thread.isReady = true - workersReadyWG.Done() - // TODO: we can also store all threads assigned to the worker if needed -} - //export go_before_worker_script func go_before_worker_script(threadIndex C.uintptr_t) *C.char { thread := phpThreads[threadIndex] @@ -246,32 +121,37 @@ func go_before_worker_script(threadIndex C.uintptr_t) *C.char { // if we are done, exit the loop that restarts the worker script if workersAreDone.Load() { - return nil - } + return nil + } + + // if we are restarting the worker, reset the exponential failure backoff + thread.backoff.reset() metrics.StartWorker(worker.fileName) - // Create main dummy request - r, err := http.NewRequest(http.MethodGet, filepath.Base(worker.fileName), nil) - if err != nil { - panic(err) - } - - r, err = NewRequestWithContext( - r, - WithRequestDocumentRoot(filepath.Dir(worker.fileName), false), - WithRequestPreparedEnv(worker.env), - ) - if err != nil { - panic(err) - } - thread.mainRequest = r - if c := logger.Check(zapcore.DebugLevel, "starting"); c != nil { - c.Write(zap.String("worker", worker.fileName), zap.Int("num", worker.num)) - } - - if err := updateServerContext(r, true, false); err != nil { - panic(err) - } + // Create a dummy request to set up the worker + r, err := http.NewRequest(http.MethodGet, filepath.Base(worker.fileName), nil) + if err != nil { + panic(err) + } + + r, err = NewRequestWithContext( + r, + WithRequestDocumentRoot(filepath.Dir(worker.fileName), false), + WithRequestPreparedEnv(worker.env), + ) + if err != nil { + panic(err) + } + + if err := updateServerContext(r, true, false); err != nil { + panic(err) + } + + thread.mainRequest = r + if c := logger.Check(zapcore.DebugLevel, "starting"); c != nil { + c.Write(zap.String("worker", worker.fileName), zap.Int("num", worker.num)) + } + return C.CString(worker.fileName) } @@ -282,34 +162,42 @@ func go_after_worker_script(threadIndex C.uintptr_t, exitStatus C.int) { fc.exitStatus = exitStatus if fc.exitStatus < 0 { - panic(ScriptExecutionError) - } - // on exit status 0 we just run the worker script again - if fc.exitStatus == 0 { - // TODO: make the max restart configurable - if c := logger.Check(zapcore.InfoLevel, "restarting"); c != nil { - c.Write(zap.String("worker", thread.worker.fileName)) - } - metrics.StopWorker(thread.worker.fileName, StopReasonRestart) - return - } else { - time.Sleep(1 * time.Millisecond) - logger.Error("worker script exited with non-zero status") - } - maybeCloseContext(fc) - thread.mainRequest = nil - thread.Unpin() + panic(ScriptExecutionError) + } + + defer func() { + maybeCloseContext(fc) + thread.mainRequest = nil + thread.Unpin() + }() + + // on exit status 0 we just run the worker script again + if fc.exitStatus == 0 { + // TODO: make the max restart configurable + metrics.StopWorker(thread.worker.fileName, StopReasonRestart) + + if c := logger.Check(zapcore.InfoLevel, "restarting"); c != nil { + c.Write(zap.String("worker", thread.worker.fileName)) + } + return + } + + // on exit status 1 we apply an exponential backoff when restarting + metrics.StopWorker(thread.worker.fileName, StopReasonCrash) + thread.backoff.trigger(func(failureCount int) { + // if we end up here, the worker has not been up for backoff*2 + // this is probably due to a syntax error or another fatal error + if !watcherIsEnabled { + panic(fmt.Errorf("workers %q: too many consecutive failures", thread.worker.fileName)) + } + logger.Warn("many consecutive worker failures", zap.String("worker", thread.worker.fileName), zap.Int("failures", failureCount)) + }) } //export go_frankenphp_worker_handle_request_start func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { thread := phpThreads[threadIndex] - - if !thread.isReady { - thread.isReady = true - workersReadyWG.Done() - metrics.ReadyWorker(thread.worker.fileName) - } + thread.setReadyForRequests() if c := logger.Check(zapcore.DebugLevel, "waiting for request"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) From 7d2ab8cc99af0bd992be3cdabf8c190e7768f29f Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 14:28:52 +0100 Subject: [PATCH 04/34] Initial working implementation. --- frankenphp.go | 2 -- php_threads.go | 6 +++-- php_threads_test.go | 60 +++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 64 insertions(+), 4 deletions(-) create mode 100644 php_threads_test.go diff --git a/frankenphp.go b/frankenphp.go index df8d99af64..765c487843 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -64,7 +64,6 @@ var ( ScriptExecutionError = errors.New("error during PHP script execution") requestChan chan *http.Request - done chan struct{} loggerMu sync.RWMutex logger *zap.Logger @@ -330,7 +329,6 @@ func Init(options ...Option) error { logger.Warn(`ZTS is not enabled, only 1 thread will be available, recompile PHP using the "--enable-zts" configuration option or performance will be degraded`) } - done = make(chan struct{}) requestChan = make(chan *http.Request) if err := initPHPThreads(opt.numThreads); err != nil { return err diff --git a/php_threads.go b/php_threads.go index 55e19f53ff..14718ed41a 100644 --- a/php_threads.go +++ b/php_threads.go @@ -12,12 +12,14 @@ var ( phpThreads []*phpThread terminationWG sync.WaitGroup mainThreadShutdownWG sync.WaitGroup - threadsReadyWG sync.WaitGroup + threadsReadyWG sync.WaitGroup shutdownWG sync.WaitGroup + done chan struct{} ) // reserve a fixed number of PHP threads on the go side func initPHPThreads(numThreads int) error { + done = make(chan struct{}) phpThreads = make([]*phpThread, numThreads) for i := 0; i < numThreads; i++ { phpThreads[i] = &phpThread{threadIndex: i} @@ -28,9 +30,9 @@ func initPHPThreads(numThreads int) error { func drainPHPThreads() { close(done) shutdownWG.Wait() - phpThreads = nil mainThreadShutdownWG.Done() terminationWG.Wait() + phpThreads = nil } func startMainThread(numThreads int) error { diff --git a/php_threads_test.go b/php_threads_test.go new file mode 100644 index 0000000000..912485309f --- /dev/null +++ b/php_threads_test.go @@ -0,0 +1,60 @@ +package frankenphp + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap" +) + +func ATestStartAndStopTheMainThread(t *testing.T) { + logger = zap.NewNop() + initPHPThreads(1) // reserve 1 thread + + assert.Equal(t, 1, len(phpThreads)) + assert.Equal(t, 0, phpThreads[0].threadIndex) + assert.False(t, phpThreads[0].isActive) + assert.False(t, phpThreads[0].isReady) + assert.Nil(t, phpThreads[0].worker) + + drainPHPThreads() + assert.Nil(t, phpThreads) +} + +func ATestStartAndStopARegularThread(t *testing.T) { + logger = zap.NewNop() // the logger needs to not be nil + initPHPThreads(1) // reserve 1 thread + + startNewPHPThread() + threadsReadyWG.Wait() + + assert.Equal(t, 1, len(phpThreads)) + assert.True(t, phpThreads[0].isActive) + assert.True(t, phpThreads[0].isReady) + assert.Nil(t, phpThreads[0].worker) + + drainPHPThreads() + assert.Nil(t, phpThreads) +} + +func ATestStartAndStopAWorkerThread(t *testing.T) { + logger = zap.NewNop() // the logger needs to not be nil + initPHPThreads(1) // reserve 1 thread + + initWorkers([]workerOpt{workerOpt { + fileName: "testdata/worker.php", + num: 1, + env: make(map[string]string, 0), + watch: make([]string, 0), + }}) + threadsReadyWG.Wait() + + assert.Equal(t, 1, len(phpThreads)) + assert.True(t, phpThreads[0].isActive) + assert.True(t, phpThreads[0].isReady) + assert.NotNil(t, phpThreads[0].worker) + + drainPHPThreads() + assert.Nil(t, phpThreads) +} + From f7e7d41f8766e7c64f6f1624957300c3f4b41f2f Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 21:27:43 +0100 Subject: [PATCH 05/34] Refactors php threads to take callbacks. --- frankenphp.c | 67 +++++++------------------- frankenphp.go | 37 +++++++-------- frankenphp.h | 1 - php_thread.go | 65 ++++++++++++++++++++++---- php_threads.go | 57 +++-------------------- php_threads_test.go | 111 ++++++++++++++++++++++++++++++++------------ testdata/sleep.php | 4 ++ worker.go | 61 +++++++++++++++--------- 8 files changed, 217 insertions(+), 186 deletions(-) create mode 100644 testdata/sleep.php diff --git a/frankenphp.c b/frankenphp.c index 93c283e3d8..988404e812 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -838,7 +838,7 @@ static void set_thread_name(char *thread_name) { #endif } -static void init_php_thread(void *arg) { +static void *php_thread(void *arg) { thread_index = (uintptr_t)arg; char thread_name[16] = {0}; snprintf(thread_name, 16, "php-%" PRIxPTR, thread_index); @@ -851,44 +851,20 @@ static void init_php_thread(void *arg) { ZEND_TSRMLS_CACHE_UPDATE(); #endif #endif - local_ctx = malloc(sizeof(frankenphp_server_context)); -} -static void shutdown_php_thread(void) { - free(local_ctx); - local_ctx = NULL; -#ifdef ZTS - ts_free_thread(); -#endif -} -static void *php_thread(void *arg) { - init_php_thread(arg); + go_frankenphp_on_thread_startup(thread_index); - // handle requests until the channel is closed - while (go_handle_request(thread_index)) { + // perform work until go signals to stop + while (go_frankenphp_on_thread_work(thread_index)) { } - shutdown_php_thread(); - go_shutdown_php_thread(thread_index); - return NULL; -} +#ifdef ZTS + ts_free_thread(); +#endif -static void *php_worker_thread(void *arg) { - init_php_thread(arg); - - // run the loop that executes the worker script - while (true) { - char *script_name = go_before_worker_script(thread_index); - if (script_name == NULL) { - break; - } - int exit_status = frankenphp_execute_script(script_name); - go_after_worker_script(thread_index, exit_status); - } + go_frankenphp_on_thread_shutdown(thread_index); - shutdown_php_thread(); - go_shutdown_worker_thread(thread_index); return NULL; } @@ -940,7 +916,7 @@ static void *php_main(void *arg) { frankenphp_sapi_module.startup(&frankenphp_sapi_module); - go_main_thread_is_ready(); + go_frankenphp_main_thread_is_ready(); /* channel closed, shutdown gracefully */ frankenphp_sapi_module.shutdown(&frankenphp_sapi_module); @@ -956,7 +932,7 @@ static void *php_main(void *arg) { frankenphp_sapi_module.ini_entries = NULL; } #endif - go_shutdown_main_thread(); + go_frankenphp_shutdown_main_thread(); return NULL; } @@ -970,22 +946,13 @@ int frankenphp_new_main_thread(int num_threads) { return pthread_detach(thread); } -int frankenphp_new_worker_thread(uintptr_t thread_index){ - pthread_t thread; - if (pthread_create(&thread, NULL, &php_worker_thread, (void *)thread_index) != 0){ - return 1; - } - pthread_detach(thread); - return 0; -} - -int frankenphp_new_php_thread(uintptr_t thread_index){ - pthread_t thread; - if (pthread_create(&thread, NULL, &php_thread, (void *)thread_index) != 0){ - return 1; - } - pthread_detach(thread); - return 0; +int frankenphp_new_php_thread(uintptr_t thread_index) { + pthread_t thread; + if (pthread_create(&thread, NULL, &php_thread, (void *)thread_index) != 0) { + return 1; + } + pthread_detach(thread); + return 0; } int frankenphp_request_startup() { diff --git a/frankenphp.go b/frankenphp.go index 765c487843..53bc9b2c3e 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -340,7 +340,9 @@ func Init(options ...Option) error { } for i := 0; i < opt.numThreads-totalWorkers; i++ { - if err := startNewPHPThread(); err != nil { + thread := getInactivePHPThread() + thread.onWork = handleRequest + if err := thread.run(); err != nil { return err } } @@ -569,16 +571,12 @@ func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string return true, value // Return 1 to indicate success } -//export go_handle_request -func go_handle_request(threadIndex C.uintptr_t) bool { - thread := phpThreads[threadIndex] - thread.setReadyForRequests() +func handleRequest(thread *phpThread) bool { select { case <-done: return false case r := <-requestChan: - thread := phpThreads[threadIndex] thread.mainRequest = r fc, ok := FromContext(r.Context()) @@ -595,11 +593,7 @@ func go_handle_request(threadIndex C.uintptr_t) bool { panic(err) } - // scriptFilename is freed in frankenphp_execute_script() - fc.exitStatus = C.frankenphp_execute_script(C.CString(fc.scriptFilename)) - if fc.exitStatus < 0 { - panic(ScriptExecutionError) - } + fc.exitStatus = executeScriptCGI(fc.scriptFilename) return true } @@ -880,6 +874,15 @@ func go_log(message *C.char, level C.int) { } } +func executeScriptCGI(script string) C.int { + // scriptFilename is freed in frankenphp_execute_script() + exitStatus := C.frankenphp_execute_script(C.CString(script)) + if exitStatus < 0 { + panic(ScriptExecutionError) + } + return exitStatus +} + // ExecuteScriptCLI executes the PHP script passed as parameter. // It returns the exit status code of the script. func ExecuteScriptCLI(script string, args []string) int { @@ -907,19 +910,11 @@ func freeArgs(argv []*C.char) { } } -func executePHPFunction(functionName string) { +func executePHPFunction(functionName string) bool { cFunctionName := C.CString(functionName) defer C.free(unsafe.Pointer(cFunctionName)) success := C.frankenphp_execute_php_function(cFunctionName) - if success == 1 { - if c := logger.Check(zapcore.DebugLevel, "php function call successful"); c != nil { - c.Write(zap.String("function", functionName)) - } - } else { - if c := logger.Check(zapcore.ErrorLevel, "php function call failed"); c != nil { - c.Write(zap.String("function", functionName)) - } - } + return success == 1 } diff --git a/frankenphp.h b/frankenphp.h index 7470ba00e4..38d408fe6b 100644 --- a/frankenphp.h +++ b/frankenphp.h @@ -42,7 +42,6 @@ frankenphp_config frankenphp_get_config(); int frankenphp_new_main_thread(int num_threads); int frankenphp_new_php_thread(uintptr_t thread_index); -int frankenphp_new_worker_thread(uintptr_t thread_index); int frankenphp_update_server_context( bool create, bool has_main_request, bool has_active_request, diff --git a/php_thread.go b/php_thread.go index 811c7a6777..3091077367 100644 --- a/php_thread.go +++ b/php_thread.go @@ -1,8 +1,11 @@ package frankenphp // #include +// #include +// #include "frankenphp.h" import "C" import ( + "fmt" "net/http" "runtime" ) @@ -13,10 +16,13 @@ type phpThread struct { mainRequest *http.Request workerRequest *http.Request worker *worker - isActive bool // whether the thread is currently running - isReady bool // whether the thread is ready to accept requests - threadIndex int // the index of the thread in the phpThreads slice - backoff *exponentialBackoff // backoff for worker failures + isActive bool // whether the thread is currently running + isReady bool // whether the thread is ready to accept requests + threadIndex int // the index of the thread in the phpThreads slice + onStartup func(*phpThread) // the function to run on startup + onWork func(*phpThread) bool // the function to run in the thread + onShutdown func(*phpThread) // the function to run after shutdown + backoff *exponentialBackoff // backoff for worker failures } func (thread phpThread) getActiveRequest() *http.Request { @@ -27,16 +33,55 @@ func (thread phpThread) getActiveRequest() *http.Request { return thread.mainRequest } -func (thread *phpThread) setReadyForRequests() { +func (thread *phpThread) run() error { + if thread.isActive { + return fmt.Errorf("thread is already running %d", thread.threadIndex) + } + threadsReadyWG.Add(1) + shutdownWG.Add(1) + thread.isActive = true + if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("error creating thread %d", thread.threadIndex) + } + return nil +} + +func (thread *phpThread) setReady() { if thread.isReady { return } - thread.isReady = true - threadsReadyWG.Done() - if thread.worker != nil { - metrics.ReadyWorker(thread.worker.fileName) - } + thread.isReady = true + threadsReadyWG.Done() + if thread.worker != nil { + metrics.ReadyWorker(thread.worker.fileName) + } } +//export go_frankenphp_on_thread_startup +func go_frankenphp_on_thread_startup(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + thread.isReady = true + if thread.onStartup != nil { + thread.onStartup(thread) + } + threadsReadyWG.Done() +} + +//export go_frankenphp_on_thread_work +func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { + thread := phpThreads[threadIndex] + return C.bool(thread.onWork(thread)) +} +//export go_frankenphp_on_thread_shutdown +func go_frankenphp_on_thread_shutdown(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + thread.isActive = false + thread.isReady = false + thread.Unpin() + if thread.onShutdown != nil { + thread.onShutdown(thread) + } + shutdownWG.Done() +} diff --git a/php_threads.go b/php_threads.go index 14718ed41a..137eb4ee84 100644 --- a/php_threads.go +++ b/php_threads.go @@ -4,7 +4,6 @@ package frankenphp // #include "frankenphp.h" import "C" import ( - "fmt" "sync" ) @@ -14,7 +13,7 @@ var ( mainThreadShutdownWG sync.WaitGroup threadsReadyWG sync.WaitGroup shutdownWG sync.WaitGroup - done chan struct{} + done chan struct{} ) // reserve a fixed number of PHP threads on the go side @@ -46,32 +45,7 @@ func startMainThread(numThreads int) error { return nil } -func startNewPHPThread() error { - threadsReadyWG.Add(1) - shutdownWG.Add(1) - thread := getInactiveThread() - thread.isActive = true - if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("error creating thread %d", thread.threadIndex) - } - return nil -} - -func startNewWorkerThread(worker *worker) error { - threadsReadyWG.Add(1) - workerShutdownWG.Add(1) - thread := getInactiveThread() - thread.worker = worker - thread.backoff = newExponentialBackoff() - thread.isActive = true - if C.frankenphp_new_worker_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("failed to create worker thread") - } - - return nil -} - -func getInactiveThread() *phpThread { +func getInactivePHPThread() *phpThread { for _, thread := range phpThreads { if !thread.isActive { return thread @@ -81,32 +55,13 @@ func getInactiveThread() *phpThread { return nil } -//export go_main_thread_is_ready -func go_main_thread_is_ready() { +//export go_frankenphp_main_thread_is_ready +func go_frankenphp_main_thread_is_ready() { threadsReadyWG.Done() mainThreadShutdownWG.Wait() } -//export go_shutdown_main_thread -func go_shutdown_main_thread() { +//export go_frankenphp_shutdown_main_thread +func go_frankenphp_shutdown_main_thread() { terminationWG.Done() } - -//export go_shutdown_php_thread -func go_shutdown_php_thread(threadIndex C.uintptr_t) { - thread := phpThreads[threadIndex] - thread.Unpin() - thread.isActive = false - thread.isReady = false - shutdownWG.Done() -} - -//export go_shutdown_worker_thread -func go_shutdown_worker_thread(threadIndex C.uintptr_t) { - thread := phpThreads[threadIndex] - thread.Unpin() - thread.isActive = false - thread.isReady = false - thread.worker = nil - workerShutdownWG.Done() -} diff --git a/php_threads_test.go b/php_threads_test.go index 912485309f..837486054e 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -1,60 +1,111 @@ package frankenphp import ( + "net/http" + "path/filepath" + "sync" + "sync/atomic" "testing" "github.com/stretchr/testify/assert" "go.uber.org/zap" ) -func ATestStartAndStopTheMainThread(t *testing.T) { - logger = zap.NewNop() +func TestStartAndStopTheMainThread(t *testing.T) { initPHPThreads(1) // reserve 1 thread assert.Equal(t, 1, len(phpThreads)) assert.Equal(t, 0, phpThreads[0].threadIndex) - assert.False(t, phpThreads[0].isActive) - assert.False(t, phpThreads[0].isReady) - assert.Nil(t, phpThreads[0].worker) + assert.False(t, phpThreads[0].isActive) + assert.False(t, phpThreads[0].isReady) + assert.Nil(t, phpThreads[0].worker) drainPHPThreads() assert.Nil(t, phpThreads) } -func ATestStartAndStopARegularThread(t *testing.T) { - logger = zap.NewNop() // the logger needs to not be nil - initPHPThreads(1) // reserve 1 thread +// We'll start 100 threads and check that their hooks work correctly +// onStartup => before the thread is ready +// onWork => while the thread is working +// onShutdown => after the thread is done +func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { + numThreads := 100 + readyThreads := atomic.Uint64{} + finishedThreads := atomic.Uint64{} + workingThreads := atomic.Uint64{} + initPHPThreads(numThreads) + + for i := 0; i < numThreads; i++ { + newThread := getInactivePHPThread() + newThread.onStartup = func(thread *phpThread) { + if thread.threadIndex == newThread.threadIndex { + readyThreads.Add(1) + } + } + newThread.onWork = func(thread *phpThread) bool { + if thread.threadIndex == newThread.threadIndex { + workingThreads.Add(1) + } + return false // stop immediately + } + newThread.onShutdown = func(thread *phpThread) { + if thread.threadIndex == newThread.threadIndex { + finishedThreads.Add(1) + } + } + newThread.run() + } - startNewPHPThread() threadsReadyWG.Wait() - assert.Equal(t, 1, len(phpThreads)) - assert.True(t, phpThreads[0].isActive) - assert.True(t, phpThreads[0].isReady) - assert.Nil(t, phpThreads[0].worker) + assert.Equal(t, numThreads, int(readyThreads.Load())) drainPHPThreads() - assert.Nil(t, phpThreads) + + assert.Equal(t, numThreads, int(workingThreads.Load())) + assert.Equal(t, numThreads, int(finishedThreads.Load())) } -func ATestStartAndStopAWorkerThread(t *testing.T) { +// This test calls sleep() 10.000 times for 1ms (completes in ~200ms) +func TestSleep10000TimesIn100Threads(t *testing.T) { logger = zap.NewNop() // the logger needs to not be nil - initPHPThreads(1) // reserve 1 thread - - initWorkers([]workerOpt{workerOpt { - fileName: "testdata/worker.php", - num: 1, - env: make(map[string]string, 0), - watch: make([]string, 0), - }}) - threadsReadyWG.Wait() + numThreads := 100 + maxExecutions := 10000 + executionMutex := sync.Mutex{} + executionCount := 0 + scriptPath, _ := filepath.Abs("./testdata/sleep.php") + initPHPThreads(numThreads) - assert.Equal(t, 1, len(phpThreads)) - assert.True(t, phpThreads[0].isActive) - assert.True(t, phpThreads[0].isReady) - assert.NotNil(t, phpThreads[0].worker) + for i := 0; i < numThreads; i++ { + newThread := getInactivePHPThread() + + // fake a request on startup (like a worker would do) + newThread.onStartup = func(thread *phpThread) { + r, _ := http.NewRequest(http.MethodGet, "sleep.php", nil) + r, _ = NewRequestWithContext(r, WithRequestDocumentRoot("/", false)) + assert.NoError(t, updateServerContext(r, true, false)) + thread.mainRequest = r + } + + // execute the php script until we reach the maxExecutions + newThread.onWork = func(thread *phpThread) bool { + executionMutex.Lock() + if executionCount >= maxExecutions { + executionMutex.Unlock() + return false + } + executionCount++ + executionMutex.Unlock() + if int(executeScriptCGI(scriptPath)) != 0 { + return false + } + + return true + } + newThread.run() + } drainPHPThreads() - assert.Nil(t, phpThreads) -} + assert.Equal(t, maxExecutions, executionCount) +} diff --git a/testdata/sleep.php b/testdata/sleep.php new file mode 100644 index 0000000000..1b1a66d029 --- /dev/null +++ b/testdata/sleep.php @@ -0,0 +1,4 @@ + Date: Sat, 2 Nov 2024 22:04:53 +0100 Subject: [PATCH 06/34] Cleanup. --- frankenphp.c | 4 ++-- frankenphp.go | 21 ++++++++------------- php_thread.go | 25 +++++++------------------ php_threads_test.go | 3 +-- worker.go | 33 ++++++++++++++++++--------------- 5 files changed, 36 insertions(+), 50 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 988404e812..42bdfca394 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -253,7 +253,7 @@ PHP_FUNCTION(frankenphp_finish_request) { /* {{{ */ php_header(); if (ctx->has_active_request) { - go_frankenphp_finish_request(thread_index, false); + go_frankenphp_finish_request_manually(thread_index); } ctx->finished = true; @@ -453,7 +453,7 @@ PHP_FUNCTION(frankenphp_handle_request) { frankenphp_worker_request_shutdown(); ctx->has_active_request = false; - go_frankenphp_finish_request(thread_index, true); + go_frankenphp_finish_worker_request(thread_index); RETURN_TRUE; } diff --git a/frankenphp.go b/frankenphp.go index 53bc9b2c3e..1db0714bc4 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -242,7 +242,7 @@ func Config() PHPConfig { // MaxThreads is internally used during tests. It is written to, but never read and may go away in the future. var MaxThreads int -func calculateMaxThreads(opt *opt) error { +func calculateMaxThreads(opt *opt) (int, int, error) { maxProcs := runtime.GOMAXPROCS(0) * 2 var numWorkers int @@ -264,13 +264,13 @@ func calculateMaxThreads(opt *opt) error { opt.numThreads = maxProcs } } else if opt.numThreads <= numWorkers { - return NotEnoughThreads + return opt.numThreads, numWorkers, NotEnoughThreads } metrics.TotalThreads(opt.numThreads) MaxThreads = opt.numThreads - return nil + return opt.numThreads, numWorkers, nil } // Init starts the PHP runtime and the configured workers. @@ -309,7 +309,7 @@ func Init(options ...Option) error { metrics = opt.metrics } - err := calculateMaxThreads(opt) + totalThreadCount, workerThreadCount, err := calculateMaxThreads(opt) if err != nil { return err } @@ -325,21 +325,16 @@ func Init(options ...Option) error { logger.Warn(`Zend Max Execution Timers are not enabled, timeouts (e.g. "max_execution_time") are disabled, recompile PHP with the "--enable-zend-max-execution-timers" configuration option to fix this issue`) } } else { - opt.numThreads = 1 + totalThreadCount = 1 logger.Warn(`ZTS is not enabled, only 1 thread will be available, recompile PHP using the "--enable-zts" configuration option or performance will be degraded`) } requestChan = make(chan *http.Request) - if err := initPHPThreads(opt.numThreads); err != nil { + if err := initPHPThreads(totalThreadCount); err != nil { return err } - totalWorkers := 0 - for _, w := range opt.workers { - totalWorkers += w.num - } - - for i := 0; i < opt.numThreads-totalWorkers; i++ { + for i := 0; i < totalThreadCount-workerThreadCount; i++ { thread := getInactivePHPThread() thread.onWork = handleRequest if err := thread.run(); err != nil { @@ -359,7 +354,7 @@ func Init(options ...Option) error { } if c := logger.Check(zapcore.InfoLevel, "FrankenPHP started 🐘"); c != nil { - c.Write(zap.String("php_version", Version().Version), zap.Int("num_threads", opt.numThreads)) + c.Write(zap.String("php_version", Version().Version), zap.Int("num_threads", totalThreadCount)) } if EmbeddedAppPath != "" { if c := logger.Check(zapcore.InfoLevel, "embedded PHP app 📦"); c != nil { diff --git a/php_thread.go b/php_thread.go index 3091077367..351b7e3566 100644 --- a/php_thread.go +++ b/php_thread.go @@ -16,11 +16,10 @@ type phpThread struct { mainRequest *http.Request workerRequest *http.Request worker *worker - isActive bool // whether the thread is currently running - isReady bool // whether the thread is ready to accept requests threadIndex int // the index of the thread in the phpThreads slice - onStartup func(*phpThread) // the function to run on startup - onWork func(*phpThread) bool // the function to run in the thread + isActive bool // whether the thread is currently running + onStartup func(*phpThread) // the function to run when ready + onWork func(*phpThread) bool // the function to run in a loop when ready onShutdown func(*phpThread) // the function to run after shutdown backoff *exponentialBackoff // backoff for worker failures } @@ -37,31 +36,22 @@ func (thread *phpThread) run() error { if thread.isActive { return fmt.Errorf("thread is already running %d", thread.threadIndex) } + if thread.onWork == nil { + return fmt.Errorf("thread.onWork must be defined %d", thread.threadIndex) + } threadsReadyWG.Add(1) shutdownWG.Add(1) thread.isActive = true if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { return fmt.Errorf("error creating thread %d", thread.threadIndex) } - return nil -} - -func (thread *phpThread) setReady() { - if thread.isReady { - return - } - thread.isReady = true - threadsReadyWG.Done() - if thread.worker != nil { - metrics.ReadyWorker(thread.worker.fileName) - } + return nil } //export go_frankenphp_on_thread_startup func go_frankenphp_on_thread_startup(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] - thread.isReady = true if thread.onStartup != nil { thread.onStartup(thread) } @@ -78,7 +68,6 @@ func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { func go_frankenphp_on_thread_shutdown(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] thread.isActive = false - thread.isReady = false thread.Unpin() if thread.onShutdown != nil { thread.onShutdown(thread) diff --git a/php_threads_test.go b/php_threads_test.go index 837486054e..3344e901ff 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -17,7 +17,6 @@ func TestStartAndStopTheMainThread(t *testing.T) { assert.Equal(t, 1, len(phpThreads)) assert.Equal(t, 0, phpThreads[0].threadIndex) assert.False(t, phpThreads[0].isActive) - assert.False(t, phpThreads[0].isReady) assert.Nil(t, phpThreads[0].worker) drainPHPThreads() @@ -66,7 +65,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { assert.Equal(t, numThreads, int(finishedThreads.Load())) } -// This test calls sleep() 10.000 times for 1ms (completes in ~200ms) +// This test calls sleep() 10.000 times for 1ms in 100 PHP threads. func TestSleep10000TimesIn100Threads(t *testing.T) { logger = zap.NewNop() // the logger needs to not be nil numThreads := 100 diff --git a/worker.go b/worker.go index bffed0327f..b4497de3e5 100644 --- a/worker.go +++ b/worker.go @@ -210,7 +210,6 @@ func afterWorkerScript(thread *phpThread, exitStatus C.int) { //export go_frankenphp_worker_handle_request_start func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { thread := phpThreads[threadIndex] - thread.setReady() if c := logger.Check(zapcore.DebugLevel, "waiting for request"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) @@ -247,28 +246,32 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { return C.bool(true) } -//export go_frankenphp_finish_request -func go_frankenphp_finish_request(threadIndex C.uintptr_t, isWorkerRequest bool) { +//export go_frankenphp_finish_worker_request +func go_frankenphp_finish_worker_request(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] r := thread.getActiveRequest() fc := r.Context().Value(contextKey).(*FrankenPHPContext) - - if isWorkerRequest { - thread.workerRequest = nil - } + thread.workerRequest = nil maybeCloseContext(fc) if c := fc.logger.Check(zapcore.DebugLevel, "request handling finished"); c != nil { - var fields []zap.Field - if isWorkerRequest { - fields = append(fields, zap.String("worker", fc.scriptFilename), zap.String("url", r.RequestURI)) - } else { - fields = append(fields, zap.String("url", r.RequestURI)) - } - - c.Write(fields...) + c.Write(zap.String("worker", fc.scriptFilename), zap.String("url", r.RequestURI)) } thread.Unpin() } + +// when frankenphp_finish_request() is directly called from PHP +// +//export go_frankenphp_finish_request_manually +func go_frankenphp_finish_request_manually(threadIndex C.uintptr_t) { + thread := phpThreads[threadIndex] + r := thread.getActiveRequest() + fc := r.Context().Value(contextKey).(*FrankenPHPContext) + maybeCloseContext(fc) + + if c := fc.logger.Check(zapcore.DebugLevel, "request handling finished"); c != nil { + c.Write(zap.String("url", r.RequestURI)) + } +} From a9857dc82eeb7cfb69a8d818aabbf0645c3bba47 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 22:18:47 +0100 Subject: [PATCH 07/34] Cleanup. --- php_threads.go | 3 +-- worker.go | 31 ++++++++++++++++++------------- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/php_threads.go b/php_threads.go index 137eb4ee84..180594d66f 100644 --- a/php_threads.go +++ b/php_threads.go @@ -51,8 +51,7 @@ func getInactivePHPThread() *phpThread { return thread } } - - return nil + panic("not enough threads reserved") } //export go_frankenphp_main_thread_is_ready diff --git a/worker.go b/worker.go index b4497de3e5..1ba3e110c2 100644 --- a/worker.go +++ b/worker.go @@ -75,16 +75,33 @@ func newWorker(o workerOpt) (*worker, error) { func startNewWorkerThread(worker *worker) error { workerShutdownWG.Add(1) thread := getInactivePHPThread() + + // onStartup => right before the thread is ready thread.onStartup = func(thread *phpThread) { thread.worker = worker metrics.ReadyWorker(worker.fileName) thread.backoff = newExponentialBackoff() } - thread.onWork = runWorkerScript + + // onWork => while the thread is working (in a loop) + thread.onWork = func(thread *phpThread) bool { + if workersAreDone.Load() { + return false + } + beforeWorkerScript(thread) + exitStatus := executeScriptCGI(thread.worker.fileName) + afterWorkerScript(thread, exitStatus) + + return true + } + + // onShutdown => after the thread is done thread.onShutdown = func(thread *phpThread) { thread.worker = nil + thread.backoff = nil workerShutdownWG.Done() } + return thread.run() } @@ -130,18 +147,6 @@ func restartWorkers(workerOpts []workerOpt) { logger.Info("workers restarted successfully") } -func runWorkerScript(thread *phpThread) bool { - // if workers are done, we stop the loop that runs the worker script - if workersAreDone.Load() { - return false - } - beforeWorkerScript(thread) - exitStatus := executeScriptCGI(thread.worker.fileName) - afterWorkerScript(thread, exitStatus) - - return true -} - func beforeWorkerScript(thread *phpThread) { worker := thread.worker From bac9555d91b5463fc315fa5936afc376d76eba47 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 22:20:54 +0100 Subject: [PATCH 08/34] Cleanup. --- php_threads_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/php_threads_test.go b/php_threads_test.go index 3344e901ff..3a074818a4 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -86,7 +86,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { thread.mainRequest = r } - // execute the php script until we reach the maxExecutions + // execute the sleep.php script until we reach maxExecutions newThread.onWork = func(thread *phpThread) bool { executionMutex.Lock() if executionCount >= maxExecutions { From a2f8d59dc6992c022027849e4cd4ee653b3729a0 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 2 Nov 2024 22:23:58 +0100 Subject: [PATCH 09/34] Cleanup. --- php_threads_test.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/php_threads_test.go b/php_threads_test.go index 3a074818a4..2b0e25e34b 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -24,9 +24,6 @@ func TestStartAndStopTheMainThread(t *testing.T) { } // We'll start 100 threads and check that their hooks work correctly -// onStartup => before the thread is ready -// onWork => while the thread is working -// onShutdown => after the thread is done func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { numThreads := 100 readyThreads := atomic.Uint64{} @@ -36,17 +33,23 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { for i := 0; i < numThreads; i++ { newThread := getInactivePHPThread() + + // onStartup => before the thread is ready newThread.onStartup = func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { readyThreads.Add(1) } } + + // onWork => while the thread is running (we stop here immediately) newThread.onWork = func(thread *phpThread) bool { if thread.threadIndex == newThread.threadIndex { workingThreads.Add(1) } return false // stop immediately } + + // onShutdown => after the thread is done newThread.onShutdown = func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { finishedThreads.Add(1) From 08254531d4f40da32e291a49619a4e75c3431a66 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sun, 3 Nov 2024 23:35:51 +0100 Subject: [PATCH 10/34] Adjusts watcher logic. --- frankenphp.go | 27 +++++++++++---------------- php_thread.go | 1 + php_threads_test.go | 2 ++ worker.go | 22 +++++++++++++--------- 4 files changed, 27 insertions(+), 25 deletions(-) diff --git a/frankenphp.go b/frankenphp.go index 1db0714bc4..b7fd240003 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -464,33 +464,28 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error fc.responseWriter = responseWriter fc.startedAt = time.Now() - isWorker := fc.responseWriter == nil isWorkerRequest := false rc := requestChan // Detect if a worker is available to handle this request - if !isWorker { - if worker, ok := workers[fc.scriptFilename]; ok { - isWorkerRequest = true - metrics.StartWorkerRequest(fc.scriptFilename) - rc = worker.requestChan - } else { - metrics.StartRequest() - } + if worker, ok := workers[fc.scriptFilename]; ok { + isWorkerRequest = true + metrics.StartWorkerRequest(fc.scriptFilename) + rc = worker.requestChan + } else { + metrics.StartRequest() } - + select { case <-done: case rc <- request: <-fc.done } - if !isWorker { - if isWorkerRequest { - metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) - } else { - metrics.StopRequest() - } + if isWorkerRequest { + metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) + } else { + metrics.StopRequest() } return nil diff --git a/php_thread.go b/php_thread.go index 351b7e3566..0b7fdf12b8 100644 --- a/php_thread.go +++ b/php_thread.go @@ -16,6 +16,7 @@ type phpThread struct { mainRequest *http.Request workerRequest *http.Request worker *worker + requestChan chan *http.Request threadIndex int // the index of the thread in the phpThreads slice isActive bool // whether the thread is currently running onStartup func(*phpThread) // the function to run when ready diff --git a/php_threads_test.go b/php_threads_test.go index 2b0e25e34b..d91fde628b 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -98,6 +98,8 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { } executionCount++ executionMutex.Unlock() + + // exit the loop and fail the test if the script fails if int(executeScriptCGI(scriptPath)) != 0 { return false } diff --git a/worker.go b/worker.go index 1ba3e110c2..d77430b1da 100644 --- a/worker.go +++ b/worker.go @@ -79,6 +79,7 @@ func startNewWorkerThread(worker *worker) error { // onStartup => right before the thread is ready thread.onStartup = func(thread *phpThread) { thread.worker = worker + thread.requestChan = chan(*http.Request) metrics.ReadyWorker(worker.fileName) thread.backoff = newExponentialBackoff() } @@ -138,7 +139,6 @@ func restartWorkersOnFileChanges(workerOpts []workerOpt) error { } func restartWorkers(workerOpts []workerOpt) { - stopWorkers() workerShutdownWG.Wait() if err := initWorkers(workerOpts); err != nil { logger.Error("failed to restart workers when watching files") @@ -226,14 +226,20 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { if c := logger.Check(zapcore.DebugLevel, "shutting down"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) } - if !executePHPFunction("opcache_reset") { - logger.Warn("opcache_reset failed") - } return C.bool(false) case r = <-thread.worker.requestChan: } + // a nil request is a signal for the worker to restart + if r == nil { + if !executePHPFunction("opcache_reset") { + logger.Warn("opcache_reset failed") + } + + return C.bool(false) + } + thread.workerRequest = r if c := logger.Check(zapcore.DebugLevel, "request handling started"); c != nil { @@ -256,23 +262,21 @@ func go_frankenphp_finish_worker_request(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] r := thread.getActiveRequest() fc := r.Context().Value(contextKey).(*FrankenPHPContext) - thread.workerRequest = nil maybeCloseContext(fc) + thread.workerRequest = nil + thread.Unpin() if c := fc.logger.Check(zapcore.DebugLevel, "request handling finished"); c != nil { c.Write(zap.String("worker", fc.scriptFilename), zap.String("url", r.RequestURI)) } - - thread.Unpin() } // when frankenphp_finish_request() is directly called from PHP // //export go_frankenphp_finish_request_manually func go_frankenphp_finish_request_manually(threadIndex C.uintptr_t) { - thread := phpThreads[threadIndex] - r := thread.getActiveRequest() + r := phpThreads[threadIndex].getActiveRequest() fc := r.Context().Value(contextKey).(*FrankenPHPContext) maybeCloseContext(fc) From 17d5cbe59f09538765c86474a297d5bf6eca4d08 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Mon, 4 Nov 2024 00:29:44 +0100 Subject: [PATCH 11/34] Adjusts the watcher logic. --- frankenphp.c | 44 ++++++++++++++++++---------------- frankenphp.go | 36 +++++++++------------------- frankenphp.h | 4 +--- php_thread.go | 9 +++---- php_threads.go | 2 +- php_threads_test.go | 4 ++-- worker.go | 57 +++++++++++++++++++++++++++++++-------------- 7 files changed, 83 insertions(+), 73 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 42bdfca394..3818ef44e5 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -965,7 +965,26 @@ int frankenphp_request_startup() { return FAILURE; } -int frankenphp_execute_script(char *file_name) { +int frankenphp_execute_php_function(const char *php_function) { + zval retval = {0}; + zend_fcall_info fci = {0}; + zend_fcall_info_cache fci_cache = {0}; + zend_string *func_name = + zend_string_init(php_function, strlen(php_function), 0); + ZVAL_STR(&fci.function_name, func_name); + fci.size = sizeof fci; + fci.retval = &retval; + int success = 0; + + zend_try { success = zend_call_function(&fci, &fci_cache) == SUCCESS; } + zend_end_try(); + + zend_string_release(func_name); + + return success; +} + +int frankenphp_execute_script(char *file_name, bool clear_op_cache) { if (frankenphp_request_startup() == FAILURE) { free(file_name); file_name = NULL; @@ -1002,6 +1021,10 @@ int frankenphp_execute_script(char *file_name) { frankenphp_free_request_context(); frankenphp_request_shutdown(); + if (clear_op_cache) { + frankenphp_execute_php_function("opcache_reset"); + } + return status; } @@ -1160,22 +1183,3 @@ int frankenphp_execute_script_cli(char *script, int argc, char **argv) { return (intptr_t)exit_status; } - -int frankenphp_execute_php_function(const char *php_function) { - zval retval = {0}; - zend_fcall_info fci = {0}; - zend_fcall_info_cache fci_cache = {0}; - zend_string *func_name = - zend_string_init(php_function, strlen(php_function), 0); - ZVAL_STR(&fci.function_name, func_name); - fci.size = sizeof fci; - fci.retval = &retval; - int success = 0; - - zend_try { success = zend_call_function(&fci, &fci_cache) == SUCCESS; } - zend_end_try(); - - zend_string_release(func_name); - - return success; -} diff --git a/frankenphp.go b/frankenphp.go index b7fd240003..d7a2797425 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -464,29 +464,24 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error fc.responseWriter = responseWriter fc.startedAt = time.Now() - isWorkerRequest := false - - rc := requestChan // Detect if a worker is available to handle this request if worker, ok := workers[fc.scriptFilename]; ok { - isWorkerRequest = true metrics.StartWorkerRequest(fc.scriptFilename) - rc = worker.requestChan - } else { - metrics.StartRequest() + worker.handleRequest(request) + <-fc.done + metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) + return nil } + + metrics.StartRequest() select { case <-done: - case rc <- request: + case requestChan <- request: <-fc.done } - if isWorkerRequest { - metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) - } else { - metrics.StopRequest() - } + metrics.StopRequest() return nil } @@ -583,7 +578,7 @@ func handleRequest(thread *phpThread) bool { panic(err) } - fc.exitStatus = executeScriptCGI(fc.scriptFilename) + fc.exitStatus = executeScriptCGI(fc.scriptFilename, false) return true } @@ -864,9 +859,9 @@ func go_log(message *C.char, level C.int) { } } -func executeScriptCGI(script string) C.int { +func executeScriptCGI(script string, clearOpCache bool) C.int { // scriptFilename is freed in frankenphp_execute_script() - exitStatus := C.frankenphp_execute_script(C.CString(script)) + exitStatus := C.frankenphp_execute_script(C.CString(script), C.bool(clearOpCache)) if exitStatus < 0 { panic(ScriptExecutionError) } @@ -899,12 +894,3 @@ func freeArgs(argv []*C.char) { C.free(unsafe.Pointer(arg)) } } - -func executePHPFunction(functionName string) bool { - cFunctionName := C.CString(functionName) - defer C.free(unsafe.Pointer(cFunctionName)) - - success := C.frankenphp_execute_php_function(cFunctionName) - - return success == 1 -} diff --git a/frankenphp.h b/frankenphp.h index 38d408fe6b..e12be3dd7a 100644 --- a/frankenphp.h +++ b/frankenphp.h @@ -50,13 +50,11 @@ int frankenphp_update_server_context( char *path_translated, char *request_uri, const char *content_type, char *auth_user, char *auth_password, int proto_num); int frankenphp_request_startup(); -int frankenphp_execute_script(char *file_name); +int frankenphp_execute_script(char *file_name, bool clear_opcache); void frankenphp_register_bulk_variables(go_string known_variables[27], php_variable *dynamic_variables, size_t size, zval *track_vars_array); int frankenphp_execute_script_cli(char *script, int argc, char **argv); -int frankenphp_execute_php_function(const char *php_function); - #endif diff --git a/php_thread.go b/php_thread.go index 0b7fdf12b8..39eeb6cf3a 100644 --- a/php_thread.go +++ b/php_thread.go @@ -7,6 +7,7 @@ import "C" import ( "fmt" "net/http" + "sync/atomic" "runtime" ) @@ -18,7 +19,7 @@ type phpThread struct { worker *worker requestChan chan *http.Request threadIndex int // the index of the thread in the phpThreads slice - isActive bool // whether the thread is currently running + isActive atomic.Bool // whether the thread is currently running onStartup func(*phpThread) // the function to run when ready onWork func(*phpThread) bool // the function to run in a loop when ready onShutdown func(*phpThread) // the function to run after shutdown @@ -34,7 +35,7 @@ func (thread phpThread) getActiveRequest() *http.Request { } func (thread *phpThread) run() error { - if thread.isActive { + if thread.isActive.Load() { return fmt.Errorf("thread is already running %d", thread.threadIndex) } if thread.onWork == nil { @@ -42,7 +43,7 @@ func (thread *phpThread) run() error { } threadsReadyWG.Add(1) shutdownWG.Add(1) - thread.isActive = true + thread.isActive.Store(true) if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { return fmt.Errorf("error creating thread %d", thread.threadIndex) } @@ -68,7 +69,7 @@ func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { //export go_frankenphp_on_thread_shutdown func go_frankenphp_on_thread_shutdown(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] - thread.isActive = false + thread.isActive.Store(false) thread.Unpin() if thread.onShutdown != nil { thread.onShutdown(thread) diff --git a/php_threads.go b/php_threads.go index 180594d66f..405e1fb55f 100644 --- a/php_threads.go +++ b/php_threads.go @@ -47,7 +47,7 @@ func startMainThread(numThreads int) error { func getInactivePHPThread() *phpThread { for _, thread := range phpThreads { - if !thread.isActive { + if !thread.isActive.Load() { return thread } } diff --git a/php_threads_test.go b/php_threads_test.go index d91fde628b..b3df3b938e 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -16,7 +16,7 @@ func TestStartAndStopTheMainThread(t *testing.T) { assert.Equal(t, 1, len(phpThreads)) assert.Equal(t, 0, phpThreads[0].threadIndex) - assert.False(t, phpThreads[0].isActive) + assert.False(t, phpThreads[0].isActive.Load()) assert.Nil(t, phpThreads[0].worker) drainPHPThreads() @@ -100,7 +100,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionMutex.Unlock() // exit the loop and fail the test if the script fails - if int(executeScriptCGI(scriptPath)) != 0 { + if int(executeScriptCGI(scriptPath, false)) != 0 { return false } diff --git a/worker.go b/worker.go index d77430b1da..0292ca919e 100644 --- a/worker.go +++ b/worker.go @@ -9,6 +9,7 @@ import ( "path/filepath" "sync" "sync/atomic" + "time" "github.com/dunglas/frankenphp/internal/watcher" "go.uber.org/zap" @@ -20,6 +21,8 @@ type worker struct { num int env PreparedEnv requestChan chan *http.Request + threads []*phpThread + threadMutex sync.RWMutex } var ( @@ -79,9 +82,12 @@ func startNewWorkerThread(worker *worker) error { // onStartup => right before the thread is ready thread.onStartup = func(thread *phpThread) { thread.worker = worker - thread.requestChan = chan(*http.Request) + thread.requestChan = make(chan *http.Request) metrics.ReadyWorker(worker.fileName) thread.backoff = newExponentialBackoff() + worker.threadMutex.Lock() + worker.threads = append(worker.threads, thread) + worker.threadMutex.Unlock() } // onWork => while the thread is working (in a loop) @@ -90,7 +96,8 @@ func startNewWorkerThread(worker *worker) error { return false } beforeWorkerScript(thread) - exitStatus := executeScriptCGI(thread.worker.fileName) + // TODO: opcache reset only if watcher is enabled + exitStatus := executeScriptCGI(thread.worker.fileName, true) afterWorkerScript(thread, exitStatus) return true @@ -119,6 +126,18 @@ func drainWorkers() { workers = make(map[string]*worker) } +// send a nil requests to workers to signal a restart +func restartWorkers() { + for _, worker := range workers { + worker.threadMutex.RLock() + for _, thread := range worker.threads { + thread.requestChan <- nil + } + worker.threadMutex.RUnlock() + } + time.Sleep(100 * time.Millisecond) // wait a bit before allowing another restart +} + func restartWorkersOnFileChanges(workerOpts []workerOpt) error { var directoriesToWatch []string for _, w := range workerOpts { @@ -128,9 +147,6 @@ func restartWorkersOnFileChanges(workerOpts []workerOpt) error { if !watcherIsEnabled { return nil } - restartWorkers := func() { - restartWorkers(workerOpts) - } if err := watcher.InitWatcher(directoriesToWatch, restartWorkers, getLogger()); err != nil { return err } @@ -138,15 +154,6 @@ func restartWorkersOnFileChanges(workerOpts []workerOpt) error { return nil } -func restartWorkers(workerOpts []workerOpt) { - workerShutdownWG.Wait() - if err := initWorkers(workerOpts); err != nil { - logger.Error("failed to restart workers when watching files") - panic(err) - } - logger.Info("workers restarted successfully") -} - func beforeWorkerScript(thread *phpThread) { worker := thread.worker @@ -212,6 +219,23 @@ func afterWorkerScript(thread *phpThread, exitStatus C.int) { }) } +func (worker *worker) handleRequest(r *http.Request) { + worker.threadMutex.RLock() + // dispatch requests to all worker threads in order + for _, thread := range worker.threads { + select { + case thread.requestChan <- r: + worker.threadMutex.RUnlock() + return + default: + } + } + worker.threadMutex.RUnlock() + // if no thread was available, fan the request out to all threads + // TODO: theoretically there could be autoscaling of threads here + worker.requestChan <- r +} + //export go_frankenphp_worker_handle_request_start func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { thread := phpThreads[threadIndex] @@ -228,15 +252,12 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { } return C.bool(false) + case r = <-thread.requestChan: case r = <-thread.worker.requestChan: } // a nil request is a signal for the worker to restart if r == nil { - if !executePHPFunction("opcache_reset") { - logger.Warn("opcache_reset failed") - } - return C.bool(false) } From 09e0ca677c14c3a20e90d4b38b52edb7199ac55b Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Mon, 4 Nov 2024 20:02:47 +0100 Subject: [PATCH 12/34] Fix opcache_reset race condition. --- frankenphp.c | 44 ++++++++++++-------------- frankenphp.go | 19 +++++++----- frankenphp.h | 3 +- php_threads_test.go | 2 +- worker.go | 76 +++++++++++++++++++++------------------------ 5 files changed, 70 insertions(+), 74 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 3818ef44e5..42bdfca394 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -965,26 +965,7 @@ int frankenphp_request_startup() { return FAILURE; } -int frankenphp_execute_php_function(const char *php_function) { - zval retval = {0}; - zend_fcall_info fci = {0}; - zend_fcall_info_cache fci_cache = {0}; - zend_string *func_name = - zend_string_init(php_function, strlen(php_function), 0); - ZVAL_STR(&fci.function_name, func_name); - fci.size = sizeof fci; - fci.retval = &retval; - int success = 0; - - zend_try { success = zend_call_function(&fci, &fci_cache) == SUCCESS; } - zend_end_try(); - - zend_string_release(func_name); - - return success; -} - -int frankenphp_execute_script(char *file_name, bool clear_op_cache) { +int frankenphp_execute_script(char *file_name) { if (frankenphp_request_startup() == FAILURE) { free(file_name); file_name = NULL; @@ -1021,10 +1002,6 @@ int frankenphp_execute_script(char *file_name, bool clear_op_cache) { frankenphp_free_request_context(); frankenphp_request_shutdown(); - if (clear_op_cache) { - frankenphp_execute_php_function("opcache_reset"); - } - return status; } @@ -1183,3 +1160,22 @@ int frankenphp_execute_script_cli(char *script, int argc, char **argv) { return (intptr_t)exit_status; } + +int frankenphp_execute_php_function(const char *php_function) { + zval retval = {0}; + zend_fcall_info fci = {0}; + zend_fcall_info_cache fci_cache = {0}; + zend_string *func_name = + zend_string_init(php_function, strlen(php_function), 0); + ZVAL_STR(&fci.function_name, func_name); + fci.size = sizeof fci; + fci.retval = &retval; + int success = 0; + + zend_try { success = zend_call_function(&fci, &fci_cache) == SUCCESS; } + zend_end_try(); + + zend_string_release(func_name); + + return success; +} diff --git a/frankenphp.go b/frankenphp.go index d7a2797425..1688a2eb40 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -349,10 +349,6 @@ func Init(options ...Option) error { // wait for all regular and worker threads to be ready for requests threadsReadyWG.Wait() - if err := restartWorkersOnFileChanges(opt.workers); err != nil { - return err - } - if c := logger.Check(zapcore.InfoLevel, "FrankenPHP started 🐘"); c != nil { c.Write(zap.String("php_version", Version().Version), zap.Int("num_threads", totalThreadCount)) } @@ -474,7 +470,7 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error } metrics.StartRequest() - + select { case <-done: case requestChan <- request: @@ -578,7 +574,7 @@ func handleRequest(thread *phpThread) bool { panic(err) } - fc.exitStatus = executeScriptCGI(fc.scriptFilename, false) + fc.exitStatus = executeScriptCGI(fc.scriptFilename) return true } @@ -859,9 +855,9 @@ func go_log(message *C.char, level C.int) { } } -func executeScriptCGI(script string, clearOpCache bool) C.int { +func executeScriptCGI(script string) C.int { // scriptFilename is freed in frankenphp_execute_script() - exitStatus := C.frankenphp_execute_script(C.CString(script), C.bool(clearOpCache)) + exitStatus := C.frankenphp_execute_script(C.CString(script)) if exitStatus < 0 { panic(ScriptExecutionError) } @@ -894,3 +890,10 @@ func freeArgs(argv []*C.char) { C.free(unsafe.Pointer(arg)) } } + +func executePHPFunction(functionName string) bool { + cFunctionName := C.CString(functionName) + defer C.free(unsafe.Pointer(cFunctionName)) + + return C.frankenphp_execute_php_function(cFunctionName) == 1 +} diff --git a/frankenphp.h b/frankenphp.h index e12be3dd7a..b903148dcb 100644 --- a/frankenphp.h +++ b/frankenphp.h @@ -50,11 +50,12 @@ int frankenphp_update_server_context( char *path_translated, char *request_uri, const char *content_type, char *auth_user, char *auth_password, int proto_num); int frankenphp_request_startup(); -int frankenphp_execute_script(char *file_name, bool clear_opcache); +int frankenphp_execute_script(char *file_name); void frankenphp_register_bulk_variables(go_string known_variables[27], php_variable *dynamic_variables, size_t size, zval *track_vars_array); int frankenphp_execute_script_cli(char *script, int argc, char **argv); +int frankenphp_execute_php_function(const char *php_function); #endif diff --git a/php_threads_test.go b/php_threads_test.go index b3df3b938e..f80c6ba822 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -100,7 +100,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionMutex.Unlock() // exit the loop and fail the test if the script fails - if int(executeScriptCGI(scriptPath, false)) != 0 { + if int(executeScriptCGI(scriptPath)) != 0 { return false } diff --git a/worker.go b/worker.go index 0292ca919e..df39edb7d4 100644 --- a/worker.go +++ b/worker.go @@ -9,7 +9,6 @@ import ( "path/filepath" "sync" "sync/atomic" - "time" "github.com/dunglas/frankenphp/internal/watcher" "go.uber.org/zap" @@ -27,15 +26,20 @@ type worker struct { var ( watcherIsEnabled bool - workerShutdownWG sync.WaitGroup workersAreDone atomic.Bool workersDone chan interface{} - workers = make(map[string]*worker) + workers map[string]*worker + isRestarting atomic.Bool + workerRestartWG sync.WaitGroup + workerShutdownWG sync.WaitGroup ) func initWorkers(opt []workerOpt) error { + workers = make(map[string]*worker, len(opt)) workersDone = make(chan interface{}) workersAreDone.Store(false) + directoriesToWatch := getDirectoriesToWatch(opt) + watcherIsEnabled = len(directoriesToWatch) > 0 for _, o := range opt { worker, err := newWorker(o) @@ -49,6 +53,14 @@ func initWorkers(opt []workerOpt) error { } } + if len(directoriesToWatch) == 0 { + return nil + } + + if err := watcher.InitWatcher(directoriesToWatch, restartWorkers, getLogger()); err != nil { + return err + } + return nil } @@ -58,12 +70,6 @@ func newWorker(o workerOpt) (*worker, error) { return nil, fmt.Errorf("worker filename is invalid %q: %w", o.fileName, err) } - // if the worker already exists, return it - // it's necessary since we don't want to destroy the channels when restarting on file changes - if w, ok := workers[absFileName]; ok { - return w, nil - } - if o.env == nil { o.env = make(PreparedEnv, 1) } @@ -76,7 +82,6 @@ func newWorker(o workerOpt) (*worker, error) { } func startNewWorkerThread(worker *worker) error { - workerShutdownWG.Add(1) thread := getInactivePHPThread() // onStartup => right before the thread is ready @@ -86,8 +91,8 @@ func startNewWorkerThread(worker *worker) error { metrics.ReadyWorker(worker.fileName) thread.backoff = newExponentialBackoff() worker.threadMutex.Lock() - worker.threads = append(worker.threads, thread) - worker.threadMutex.Unlock() + worker.threads = append(worker.threads, thread) + worker.threadMutex.Unlock() } // onWork => while the thread is working (in a loop) @@ -95,9 +100,12 @@ func startNewWorkerThread(worker *worker) error { if workersAreDone.Load() { return false } + if watcherIsEnabled && isRestarting.Load() { + workerShutdownWG.Done() + workerRestartWG.Wait() + } beforeWorkerScript(thread) - // TODO: opcache reset only if watcher is enabled - exitStatus := executeScriptCGI(thread.worker.fileName, true) + exitStatus := executeScriptCGI(thread.worker.fileName) afterWorkerScript(thread, exitStatus) return true @@ -107,7 +115,6 @@ func startNewWorkerThread(worker *worker) error { thread.onShutdown = func(thread *phpThread) { thread.worker = nil thread.backoff = nil - workerShutdownWG.Done() } return thread.run() @@ -122,36 +129,27 @@ func drainWorkers() { watcher.DrainWatcher() watcherIsEnabled = false stopWorkers() - workerShutdownWG.Wait() - workers = make(map[string]*worker) } -// send a nil requests to workers to signal a restart func restartWorkers() { + workerRestartWG.Add(1) for _, worker := range workers { - worker.threadMutex.RLock() - for _, thread := range worker.threads { - thread.requestChan <- nil - } - worker.threadMutex.RUnlock() + workerShutdownWG.Add(worker.num) } - time.Sleep(100 * time.Millisecond) // wait a bit before allowing another restart + isRestarting.Store(true) + close(workersDone) + workerShutdownWG.Wait() + workersDone = make(chan interface{}) + isRestarting.Store(false) + workerRestartWG.Done() } -func restartWorkersOnFileChanges(workerOpts []workerOpt) error { - var directoriesToWatch []string +func getDirectoriesToWatch(workerOpts []workerOpt) []string { + directoriesToWatch := []string{} for _, w := range workerOpts { directoriesToWatch = append(directoriesToWatch, w.watch...) } - watcherIsEnabled = len(directoriesToWatch) > 0 - if !watcherIsEnabled { - return nil - } - if err := watcher.InitWatcher(directoriesToWatch, restartWorkers, getLogger()); err != nil { - return err - } - - return nil + return directoriesToWatch } func beforeWorkerScript(thread *phpThread) { @@ -250,17 +248,15 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { if c := logger.Check(zapcore.DebugLevel, "shutting down"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) } + if isRestarting.Load() && !executePHPFunction("opcache_reset") { + logger.Error("failed to call opcache_reset") + } return C.bool(false) case r = <-thread.requestChan: case r = <-thread.worker.requestChan: } - // a nil request is a signal for the worker to restart - if r == nil { - return C.bool(false) - } - thread.workerRequest = r if c := logger.Check(zapcore.DebugLevel, "request handling started"); c != nil { From 7f13ada3e6a451f3310f3b24f99afb94dcfb4896 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Mon, 4 Nov 2024 20:33:37 +0100 Subject: [PATCH 13/34] Fixing merge conflicts and formatting. --- php_thread.go | 24 ++++++++++++------------ php_threads_test.go | 2 +- worker.go | 31 +++++++++++++++---------------- 3 files changed, 28 insertions(+), 29 deletions(-) diff --git a/php_thread.go b/php_thread.go index f5771209d5..9692f22434 100644 --- a/php_thread.go +++ b/php_thread.go @@ -8,24 +8,24 @@ import "C" import ( "fmt" "net/http" - "sync/atomic" "runtime" + "sync/atomic" "unsafe" ) type phpThread struct { runtime.Pinner - mainRequest *http.Request - workerRequest *http.Request - worker *worker - requestChan chan *http.Request - threadIndex int // the index of the thread in the phpThreads slice - isActive atomic.Bool // whether the thread is currently running - onStartup func(*phpThread) // the function to run when ready - onWork func(*phpThread) bool // the function to run in a loop when ready - onShutdown func(*phpThread) // the function to run after shutdown - backoff *exponentialBackoff // backoff for worker failures + mainRequest *http.Request + workerRequest *http.Request + worker *worker + requestChan chan *http.Request + threadIndex int // the index of the thread in the phpThreads slice + isActive atomic.Bool // whether the thread is currently running + onStartup func(*phpThread) // the function to run when ready + onWork func(*phpThread) bool // the function to run in a loop when ready + onShutdown func(*phpThread) // the function to run after shutdown + backoff *exponentialBackoff // backoff for worker failures knownVariableKeys map[string]*C.zend_string } @@ -64,7 +64,7 @@ func (thread *phpThread) pinString(s string) *C.char { // C strings must be null-terminated func (thread *phpThread) pinCString(s string) *C.char { - return thread.pinString(s+"\x00") + return thread.pinString(s + "\x00") } //export go_frankenphp_on_thread_startup diff --git a/php_threads_test.go b/php_threads_test.go index f80c6ba822..c33932f7d0 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -85,7 +85,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { newThread.onStartup = func(thread *phpThread) { r, _ := http.NewRequest(http.MethodGet, "sleep.php", nil) r, _ = NewRequestWithContext(r, WithRequestDocumentRoot("/", false)) - assert.NoError(t, updateServerContext(r, true, false)) + assert.NoError(t, updateServerContext(thread, r, true, false)) thread.mainRequest = r } diff --git a/worker.go b/worker.go index f74fd3eeea..360cd4953b 100644 --- a/worker.go +++ b/worker.go @@ -25,13 +25,13 @@ type worker struct { } var ( - watcherIsEnabled bool - workersAreDone atomic.Bool - workersDone chan interface{} - workers map[string]*worker - isRestarting atomic.Bool - workerRestartWG sync.WaitGroup - workerShutdownWG sync.WaitGroup + workers map[string]*worker + workersDone chan interface{} + watcherIsEnabled bool + workersAreDone atomic.Bool + workersAreRestarting atomic.Bool + workerRestartWG sync.WaitGroup + workerShutdownWG sync.WaitGroup ) func initWorkers(opt []workerOpt) error { @@ -101,7 +101,7 @@ func startNewWorkerThread(worker *worker) error { if workersAreDone.Load() { return false } - if watcherIsEnabled && isRestarting.Load() { + if watcherIsEnabled && workersAreRestarting.Load() { workerShutdownWG.Done() workerRestartWG.Wait() } @@ -134,15 +134,15 @@ func drainWorkers() { func restartWorkers() { workerRestartWG.Add(1) + defer workerRestartWG.Done() for _, worker := range workers { workerShutdownWG.Add(worker.num) } - isRestarting.Store(true) + workersAreRestarting.Store(true) close(workersDone) workerShutdownWG.Wait() workersDone = make(chan interface{}) - isRestarting.Store(false) - workerRestartWG.Done() + workersAreRestarting.Store(false) } func getDirectoriesToWatch(workerOpts []workerOpt) []string { @@ -175,7 +175,7 @@ func beforeWorkerScript(thread *phpThread) { panic(err) } - if err := updateServerContext(r, true, false); err != nil { + if err := updateServerContext(thread, r, true, false); err != nil { panic(err) } @@ -249,14 +249,15 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { if c := logger.Check(zapcore.DebugLevel, "shutting down"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) } - if isRestarting.Load() && !executePHPFunction("opcache_reset") { + + // execute opcache_reset if the restart was triggered by the watcher + if watcherIsEnabled && workersAreRestarting.Load() && !executePHPFunction("opcache_reset") { logger.Error("failed to call opcache_reset") } return C.bool(false) case r = <-thread.requestChan: case r = <-thread.worker.requestChan: - case r = <-thread.requestChan: } thread.workerRequest = r @@ -307,6 +308,4 @@ func go_frankenphp_finish_request_manually(threadIndex C.uintptr_t) { if c := fc.logger.Check(zapcore.DebugLevel, "request handling finished"); c != nil { c.Write(zap.String("url", r.RequestURI)) } - - thread.Unpin() } From 13fb4bb729d143f625638320e877f6d0433143bf Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 11:39:51 +0100 Subject: [PATCH 14/34] Prevents overlapping of TSRM reservation and script execution. --- frankenphp.c | 6 +- frankenphp.go | 16 ++--- php_thread.go | 81 ++++++++++++++--------- php_threads.go | 24 ++++++- php_threads_test.go | 153 +++++++++++++++++++++++++++++--------------- worker.go | 73 +++++++++------------ 6 files changed, 210 insertions(+), 143 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 7403cabb37..79bcfb989e 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -822,8 +822,6 @@ static void *php_thread(void *arg) { cfg_get_string("filter.default", &default_filter); should_filter_var = default_filter != NULL; - go_frankenphp_on_thread_startup(thread_index); - // perform work until go signals to stop while (go_frankenphp_on_thread_work(thread_index)) { } @@ -853,13 +851,11 @@ static void *php_main(void *arg) { exit(EXIT_FAILURE); } - intptr_t num_threads = (intptr_t)arg; - set_thread_name("php-main"); #ifdef ZTS #if (PHP_VERSION_ID >= 80300) - php_tsrm_startup_ex(num_threads); + php_tsrm_startup_ex((intptr_t)arg); #else php_tsrm_startup(); #endif diff --git a/frankenphp.go b/frankenphp.go index a405d0af3f..a9cab8ced9 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -336,19 +336,13 @@ func Init(options ...Option) error { for i := 0; i < totalThreadCount-workerThreadCount; i++ { thread := getInactivePHPThread() - thread.onWork = handleRequest - if err := thread.run(); err != nil { - return err - } + thread.setHooks(nil, handleRequest, nil) } if err := initWorkers(opt.workers); err != nil { return err } - // wait for all regular and worker threads to be ready for requests - threadsReadyWG.Wait() - if c := logger.Check(zapcore.InfoLevel, "FrankenPHP started 🐘"); c != nil { c.Write(zap.String("php_version", Version().Version), zap.Int("num_threads", totalThreadCount)) } @@ -556,10 +550,10 @@ func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string return true, value // Return 1 to indicate success } -func handleRequest(thread *phpThread) bool { +func handleRequest(thread *phpThread) { select { case <-done: - return false + return case r := <-requestChan: thread.mainRequest = r @@ -576,12 +570,10 @@ func handleRequest(thread *phpThread) bool { if err := updateServerContext(thread, r, true, false); err != nil { rejectRequest(fc.responseWriter, err.Error()) - return true + return } fc.exitStatus = executeScriptCGI(fc.scriptFilename) - - return true } } diff --git a/php_thread.go b/php_thread.go index 9692f22434..d07cf4c1a6 100644 --- a/php_thread.go +++ b/php_thread.go @@ -6,7 +6,6 @@ package frankenphp // #include "frankenphp.h" import "C" import ( - "fmt" "net/http" "runtime" "sync/atomic" @@ -20,12 +19,14 @@ type phpThread struct { workerRequest *http.Request worker *worker requestChan chan *http.Request - threadIndex int // the index of the thread in the phpThreads slice - isActive atomic.Bool // whether the thread is currently running - onStartup func(*phpThread) // the function to run when ready - onWork func(*phpThread) bool // the function to run in a loop when ready - onShutdown func(*phpThread) // the function to run after shutdown - backoff *exponentialBackoff // backoff for worker failures + done chan struct{} // to signal the thread to stop the + threadIndex int // the index of the thread in the phpThreads slice + isActive atomic.Bool // whether the thread is currently running + isReady atomic.Bool // whether the thread is ready for work + onStartup func(*phpThread) // the function to run when ready + onWork func(*phpThread) // the function to run in a loop when ready + onShutdown func(*phpThread) // the function to run after shutdown + backoff *exponentialBackoff // backoff for worker failures knownVariableKeys map[string]*C.zend_string } @@ -37,21 +38,36 @@ func (thread phpThread) getActiveRequest() *http.Request { return thread.mainRequest } -func (thread *phpThread) run() error { - if thread.isActive.Load() { - return fmt.Errorf("thread is already running %d", thread.threadIndex) - } - if thread.onWork == nil { - return fmt.Errorf("thread.onWork must be defined %d", thread.threadIndex) +func (thread *phpThread) setInactive() { + thread.isActive.Store(false) + thread.onWork = func(thread *phpThread) { + thread.requestChan = make(chan *http.Request) + select { + case <-done: + case <-thread.done: + } } - threadsReadyWG.Add(1) - shutdownWG.Add(1) +} + +func (thread *phpThread) setHooks(onStartup func(*phpThread), onWork func(*phpThread), onShutdown func(*phpThread)) { thread.isActive.Store(true) - if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("error creating thread %d", thread.threadIndex) + + // to avoid race conditions, the thread sets its own hooks on startup + thread.onStartup = func(thread *phpThread) { + if thread.onShutdown != nil { + thread.onShutdown(thread) + } + thread.onStartup = onStartup + thread.onWork = onWork + thread.onShutdown = onShutdown + if thread.onStartup != nil { + thread.onStartup(thread) + } } - return nil + threadsReadyWG.Add(1) + close(thread.done) + thread.isReady.Store(false) } // Pin a string that is not null-terminated @@ -67,25 +83,32 @@ func (thread *phpThread) pinCString(s string) *C.char { return thread.pinString(s + "\x00") } -//export go_frankenphp_on_thread_startup -func go_frankenphp_on_thread_startup(threadIndex C.uintptr_t) { - thread := phpThreads[threadIndex] - if thread.onStartup != nil { - thread.onStartup(thread) - } - threadsReadyWG.Done() -} - //export go_frankenphp_on_thread_work func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { + // first check if FrankPHP is shutting down + if threadsAreDone.Load() { + return C.bool(false) + } thread := phpThreads[threadIndex] - return C.bool(thread.onWork(thread)) + + // if the thread is not ready, set it up + if !thread.isReady.Load() { + thread.isReady.Store(true) + thread.done = make(chan struct{}) + if thread.onStartup != nil { + thread.onStartup(thread) + } + threadsReadyWG.Done() + } + + // do the actual work + thread.onWork(thread) + return C.bool(true) } //export go_frankenphp_on_thread_shutdown func go_frankenphp_on_thread_shutdown(threadIndex C.uintptr_t) { thread := phpThreads[threadIndex] - thread.isActive.Store(false) thread.Unpin() if thread.onShutdown != nil { thread.onShutdown(thread) diff --git a/php_threads.go b/php_threads.go index 405e1fb55f..76f23b1739 100644 --- a/php_threads.go +++ b/php_threads.go @@ -4,7 +4,9 @@ package frankenphp // #include "frankenphp.h" import "C" import ( + "fmt" "sync" + "sync/atomic" ) var ( @@ -14,19 +16,39 @@ var ( threadsReadyWG sync.WaitGroup shutdownWG sync.WaitGroup done chan struct{} + threadsAreDone atomic.Bool ) // reserve a fixed number of PHP threads on the go side func initPHPThreads(numThreads int) error { + threadsAreDone.Store(false) done = make(chan struct{}) phpThreads = make([]*phpThread, numThreads) for i := 0; i < numThreads; i++ { phpThreads[i] = &phpThread{threadIndex: i} } - return startMainThread(numThreads) + logger.Warn("initializing main thread") + if err := startMainThread(numThreads); err != nil { + return err + } + + // initialize all threads as inactive + threadsReadyWG.Add(len(phpThreads)) + shutdownWG.Add(len(phpThreads)) + for _, thread := range phpThreads { + logger.Warn("initializing thread") + thread.setInactive() + logger.Warn("thread initialized") + if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { + return fmt.Errorf("unable to create thread %d", thread.threadIndex) + } + } + threadsReadyWG.Wait() + return nil } func drainPHPThreads() { + threadsAreDone.Store(true) close(done) shutdownWG.Wait() mainThreadShutdownWG.Done() diff --git a/php_threads_test.go b/php_threads_test.go index c33932f7d0..f745b427b5 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -12,7 +12,8 @@ import ( ) func TestStartAndStopTheMainThread(t *testing.T) { - initPHPThreads(1) // reserve 1 thread + logger = zap.NewNop() // the logger needs to not be nil + initPHPThreads(1) // reserve 1 thread assert.Equal(t, 1, len(phpThreads)) assert.Equal(t, 0, phpThreads[0].threadIndex) @@ -25,45 +26,45 @@ func TestStartAndStopTheMainThread(t *testing.T) { // We'll start 100 threads and check that their hooks work correctly func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { + logger = zap.NewNop() // the logger needs to not be nil numThreads := 100 readyThreads := atomic.Uint64{} finishedThreads := atomic.Uint64{} workingThreads := atomic.Uint64{} initPHPThreads(numThreads) + workWG := sync.WaitGroup{} + workWG.Add(numThreads) for i := 0; i < numThreads; i++ { newThread := getInactivePHPThread() - - // onStartup => before the thread is ready - newThread.onStartup = func(thread *phpThread) { - if thread.threadIndex == newThread.threadIndex { - readyThreads.Add(1) - } - } - - // onWork => while the thread is running (we stop here immediately) - newThread.onWork = func(thread *phpThread) bool { - if thread.threadIndex == newThread.threadIndex { - workingThreads.Add(1) - } - return false // stop immediately - } - - // onShutdown => after the thread is done - newThread.onShutdown = func(thread *phpThread) { - if thread.threadIndex == newThread.threadIndex { - finishedThreads.Add(1) - } - } - newThread.run() + newThread.setHooks( + // onStartup => before the thread is ready + func(thread *phpThread) { + if thread.threadIndex == newThread.threadIndex { + readyThreads.Add(1) + } + }, + // onWork => while the thread is running (we stop here immediately) + func(thread *phpThread) { + if thread.threadIndex == newThread.threadIndex { + workingThreads.Add(1) + } + workWG.Done() + newThread.setInactive() + }, + // onShutdown => after the thread is done + func(thread *phpThread) { + if thread.threadIndex == newThread.threadIndex { + finishedThreads.Add(1) + } + }, + ) } - threadsReadyWG.Wait() - - assert.Equal(t, numThreads, int(readyThreads.Load())) - + workWG.Wait() drainPHPThreads() + assert.Equal(t, numThreads, int(readyThreads.Load())) assert.Equal(t, numThreads, int(workingThreads.Load())) assert.Equal(t, numThreads, int(finishedThreads.Load())) } @@ -77,39 +78,87 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionCount := 0 scriptPath, _ := filepath.Abs("./testdata/sleep.php") initPHPThreads(numThreads) + workWG := sync.WaitGroup{} + workWG.Add(maxExecutions) for i := 0; i < numThreads; i++ { - newThread := getInactivePHPThread() + getInactivePHPThread().setHooks( + // onStartup => fake a request on startup (like a worker would do) + func(thread *phpThread) { + r, _ := http.NewRequest(http.MethodGet, "sleep.php", nil) + r, _ = NewRequestWithContext(r, WithRequestDocumentRoot("/", false)) + assert.NoError(t, updateServerContext(thread, r, true, false)) + thread.mainRequest = r + }, + // onWork => execute the sleep.php script until we reach maxExecutions + func(thread *phpThread) { + executionMutex.Lock() + if executionCount >= maxExecutions { + executionMutex.Unlock() + thread.setInactive() + return + } + executionCount++ + workWG.Done() + executionMutex.Unlock() - // fake a request on startup (like a worker would do) - newThread.onStartup = func(thread *phpThread) { - r, _ := http.NewRequest(http.MethodGet, "sleep.php", nil) - r, _ = NewRequestWithContext(r, WithRequestDocumentRoot("/", false)) - assert.NoError(t, updateServerContext(thread, r, true, false)) - thread.mainRequest = r - } + // exit the loop and fail the test if the script fails + if int(executeScriptCGI(scriptPath)) != 0 { + panic("script execution failed: " + scriptPath) + } + }, + // onShutdown => nothing to do here + nil, + ) + } - // execute the sleep.php script until we reach maxExecutions - newThread.onWork = func(thread *phpThread) bool { - executionMutex.Lock() - if executionCount >= maxExecutions { - executionMutex.Unlock() - return false - } - executionCount++ - executionMutex.Unlock() + workWG.Wait() + drainPHPThreads() - // exit the loop and fail the test if the script fails - if int(executeScriptCGI(scriptPath)) != 0 { - return false - } + assert.Equal(t, maxExecutions, executionCount) +} - return true +func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { + logger = zap.NewNop() // the logger needs to not be nil + numThreads := 100 + numConversions := 10 + startUpTypes := make([]atomic.Uint64, numConversions) + workTypes := make([]atomic.Uint64, numConversions) + shutdownTypes := make([]atomic.Uint64, numConversions) + workWG := sync.WaitGroup{} + + initPHPThreads(numThreads) + + for i := 0; i < numConversions; i++ { + workWG.Add(numThreads) + numberOfConversion := i + for j := 0; j < numThreads; j++ { + getInactivePHPThread().setHooks( + // onStartup => before the thread is ready + func(thread *phpThread) { + startUpTypes[numberOfConversion].Add(1) + }, + // onWork => while the thread is running + func(thread *phpThread) { + workTypes[numberOfConversion].Add(1) + thread.setInactive() + workWG.Done() + }, + // onShutdown => after the thread is done + func(thread *phpThread) { + shutdownTypes[numberOfConversion].Add(1) + }, + ) } - newThread.run() + workWG.Wait() } drainPHPThreads() - assert.Equal(t, maxExecutions, executionCount) + // each type of thread needs to have started, worked and stopped the same amount of times + for i := 0; i < numConversions; i++ { + assert.Equal(t, numThreads, int(startUpTypes[i].Load())) + assert.Equal(t, numThreads, int(workTypes[i].Load())) + assert.Equal(t, numThreads, int(shutdownTypes[i].Load())) + } } diff --git a/worker.go b/worker.go index 360cd4953b..953b77f3c1 100644 --- a/worker.go +++ b/worker.go @@ -28,7 +28,6 @@ var ( workers map[string]*worker workersDone chan interface{} watcherIsEnabled bool - workersAreDone atomic.Bool workersAreRestarting atomic.Bool workerRestartWG sync.WaitGroup workerShutdownWG sync.WaitGroup @@ -37,7 +36,6 @@ var ( func initWorkers(opt []workerOpt) error { workers = make(map[string]*worker, len(opt)) workersDone = make(chan interface{}) - workersAreDone.Store(false) directoriesToWatch := getDirectoriesToWatch(opt) watcherIsEnabled = len(directoriesToWatch) > 0 @@ -48,9 +46,7 @@ func initWorkers(opt []workerOpt) error { return err } for i := 0; i < worker.num; i++ { - if err := startNewWorkerThread(worker); err != nil { - return err - } + worker.startNewThread() } } @@ -82,53 +78,42 @@ func newWorker(o workerOpt) (*worker, error) { return w, nil } -func startNewWorkerThread(worker *worker) error { - thread := getInactivePHPThread() - - // onStartup => right before the thread is ready - thread.onStartup = func(thread *phpThread) { - thread.worker = worker - thread.requestChan = make(chan *http.Request) - metrics.ReadyWorker(worker.fileName) - thread.backoff = newExponentialBackoff() - worker.threadMutex.Lock() - worker.threads = append(worker.threads, thread) - worker.threadMutex.Unlock() - } - - // onWork => while the thread is working (in a loop) - thread.onWork = func(thread *phpThread) bool { - if workersAreDone.Load() { - return false - } - if watcherIsEnabled && workersAreRestarting.Load() { - workerShutdownWG.Done() - workerRestartWG.Wait() - } - beforeWorkerScript(thread) - exitStatus := executeScriptCGI(thread.worker.fileName) - afterWorkerScript(thread, exitStatus) - - return true - } - - // onShutdown => after the thread is done - thread.onShutdown = func(thread *phpThread) { - thread.worker = nil - thread.backoff = nil - } - - return thread.run() +func (worker *worker) startNewThread() { + getInactivePHPThread().setHooks( + // onStartup => right before the thread is ready + func(thread *phpThread) { + thread.worker = worker + thread.requestChan = make(chan *http.Request) + metrics.ReadyWorker(worker.fileName) + thread.backoff = newExponentialBackoff() + worker.threadMutex.Lock() + worker.threads = append(worker.threads, thread) + worker.threadMutex.Unlock() + }, + // onWork => while the thread is working (in a loop) + func(thread *phpThread) { + if watcherIsEnabled && workersAreRestarting.Load() { + workerShutdownWG.Done() + workerRestartWG.Wait() + } + beforeWorkerScript(thread) + exitStatus := executeScriptCGI(thread.worker.fileName) + afterWorkerScript(thread, exitStatus) + }, + // onShutdown => after the thread is done + func(thread *phpThread) { + thread.worker = nil + thread.backoff = nil + }, + ) } func stopWorkers() { - workersAreDone.Store(true) close(workersDone) } func drainWorkers() { watcher.DrainWatcher() - watcherIsEnabled = false stopWorkers() } From a8a00c83724281f687e6a6cb63e13714bcd802b9 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 13:07:36 +0100 Subject: [PATCH 15/34] Adjustments as suggested by @dunglas. --- frankenphp.c | 8 ++++---- frankenphp.go | 4 ++-- frankenphp.h | 2 +- php_threads.go | 4 ++-- php_threads_test.go | 6 +++--- testdata/sleep.php | 2 +- worker.go | 6 +++--- 7 files changed, 16 insertions(+), 16 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 79bcfb989e..7a357e0930 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -243,7 +243,7 @@ PHP_FUNCTION(frankenphp_finish_request) { /* {{{ */ php_header(); if (ctx->has_active_request) { - go_frankenphp_finish_request_manually(thread_index); + go_frankenphp_finish_php_request(thread_index); } ctx->finished = true; @@ -913,13 +913,13 @@ int frankenphp_new_main_thread(int num_threads) { return pthread_detach(thread); } -int frankenphp_new_php_thread(uintptr_t thread_index) { +bool frankenphp_new_php_thread(uintptr_t thread_index) { pthread_t thread; if (pthread_create(&thread, NULL, &php_thread, (void *)thread_index) != 0) { - return 1; + return false; } pthread_detach(thread); - return 0; + return true; } int frankenphp_request_startup() { diff --git a/frankenphp.go b/frankenphp.go index a9cab8ced9..ebaf955845 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -573,7 +573,7 @@ func handleRequest(thread *phpThread) { return } - fc.exitStatus = executeScriptCGI(fc.scriptFilename) + fc.exitStatus = executeScriptClassic(fc.scriptFilename) } } @@ -787,7 +787,7 @@ func go_log(message *C.char, level C.int) { } } -func executeScriptCGI(script string) C.int { +func executeScriptClassic(script string) C.int { // scriptFilename is freed in frankenphp_execute_script() exitStatus := C.frankenphp_execute_script(C.CString(script)) if exitStatus < 0 { diff --git a/frankenphp.h b/frankenphp.h index ca91fc2d42..6d2e4efe2a 100644 --- a/frankenphp.h +++ b/frankenphp.h @@ -41,7 +41,7 @@ typedef struct frankenphp_config { frankenphp_config frankenphp_get_config(); int frankenphp_new_main_thread(int num_threads); -int frankenphp_new_php_thread(uintptr_t thread_index); +bool frankenphp_new_php_thread(uintptr_t thread_index); int frankenphp_update_server_context( bool create, bool has_main_request, bool has_active_request, diff --git a/php_threads.go b/php_threads.go index 76f23b1739..07da30db0a 100644 --- a/php_threads.go +++ b/php_threads.go @@ -39,8 +39,8 @@ func initPHPThreads(numThreads int) error { logger.Warn("initializing thread") thread.setInactive() logger.Warn("thread initialized") - if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) != 0 { - return fmt.Errorf("unable to create thread %d", thread.threadIndex) + if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { + panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) } } threadsReadyWG.Wait() diff --git a/php_threads_test.go b/php_threads_test.go index f745b427b5..627947dee7 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -11,11 +11,11 @@ import ( "go.uber.org/zap" ) -func TestStartAndStopTheMainThread(t *testing.T) { +func TestStartAndStopTheMainThreadWithOneInactiveThread(t *testing.T) { logger = zap.NewNop() // the logger needs to not be nil initPHPThreads(1) // reserve 1 thread - assert.Equal(t, 1, len(phpThreads)) + assert.Len(t, phpThreads, 1) assert.Equal(t, 0, phpThreads[0].threadIndex) assert.False(t, phpThreads[0].isActive.Load()) assert.Nil(t, phpThreads[0].worker) @@ -103,7 +103,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionMutex.Unlock() // exit the loop and fail the test if the script fails - if int(executeScriptCGI(scriptPath)) != 0 { + if int(executeScriptClassic(scriptPath)) != 0 { panic("script execution failed: " + scriptPath) } }, diff --git a/testdata/sleep.php b/testdata/sleep.php index 1b1a66d029..d2c78b865d 100644 --- a/testdata/sleep.php +++ b/testdata/sleep.php @@ -1,4 +1,4 @@ after the thread is done @@ -284,8 +284,8 @@ func go_frankenphp_finish_worker_request(threadIndex C.uintptr_t) { // when frankenphp_finish_request() is directly called from PHP // -//export go_frankenphp_finish_request_manually -func go_frankenphp_finish_request_manually(threadIndex C.uintptr_t) { +//export go_frankenphp_finish_php_request +func go_frankenphp_finish_php_request(threadIndex C.uintptr_t) { r := phpThreads[threadIndex].getActiveRequest() fc := r.Context().Value(contextKey).(*FrankenPHPContext) maybeCloseContext(fc) From b4dd1382a7358cd793c508c6b6a114a0df7518a4 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 13:31:27 +0100 Subject: [PATCH 16/34] Adds error assertions. --- php_threads_test.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/php_threads_test.go b/php_threads_test.go index 627947dee7..ea31ef2877 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -13,7 +13,7 @@ import ( func TestStartAndStopTheMainThreadWithOneInactiveThread(t *testing.T) { logger = zap.NewNop() // the logger needs to not be nil - initPHPThreads(1) // reserve 1 thread + assert.NoError(t, initPHPThreads(1)) // reserve 1 thread assert.Len(t, phpThreads, 1) assert.Equal(t, 0, phpThreads[0].threadIndex) @@ -31,10 +31,11 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { readyThreads := atomic.Uint64{} finishedThreads := atomic.Uint64{} workingThreads := atomic.Uint64{} - initPHPThreads(numThreads) workWG := sync.WaitGroup{} workWG.Add(numThreads) + assert.NoError(t, initPHPThreads(numThreads)) + for i := 0; i < numThreads; i++ { newThread := getInactivePHPThread() newThread.setHooks( @@ -77,10 +78,11 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionMutex := sync.Mutex{} executionCount := 0 scriptPath, _ := filepath.Abs("./testdata/sleep.php") - initPHPThreads(numThreads) workWG := sync.WaitGroup{} workWG.Add(maxExecutions) + assert.NoError(t, initPHPThreads(numThreads)) + for i := 0; i < numThreads; i++ { getInactivePHPThread().setHooks( // onStartup => fake a request on startup (like a worker would do) @@ -118,6 +120,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { assert.Equal(t, maxExecutions, executionCount) } +// TODO: Make this test more chaotic func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { logger = zap.NewNop() // the logger needs to not be nil numThreads := 100 @@ -127,7 +130,7 @@ func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { shutdownTypes := make([]atomic.Uint64, numConversions) workWG := sync.WaitGroup{} - initPHPThreads(numThreads) + assert.NoError(t, initPHPThreads(numThreads)) for i := 0; i < numConversions; i++ { workWG.Add(numThreads) From 03f98fadb09c9585a27ee5da54905467bb531fb1 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 14:41:43 +0100 Subject: [PATCH 17/34] Adds comments. --- php_thread.go | 36 ++++++++++++++++++++++++------------ 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/php_thread.go b/php_thread.go index d07cf4c1a6..8e9232c785 100644 --- a/php_thread.go +++ b/php_thread.go @@ -15,18 +15,28 @@ import ( type phpThread struct { runtime.Pinner - mainRequest *http.Request - workerRequest *http.Request - worker *worker - requestChan chan *http.Request - done chan struct{} // to signal the thread to stop the - threadIndex int // the index of the thread in the phpThreads slice - isActive atomic.Bool // whether the thread is currently running - isReady atomic.Bool // whether the thread is ready for work - onStartup func(*phpThread) // the function to run when ready - onWork func(*phpThread) // the function to run in a loop when ready - onShutdown func(*phpThread) // the function to run after shutdown - backoff *exponentialBackoff // backoff for worker failures + mainRequest *http.Request + workerRequest *http.Request + requestChan chan *http.Request + worker *worker + + // the index in the phpThreads slice + threadIndex int + // whether the thread has work assigned to it + isActive atomic.Bool + // whether the thread is ready for work + isReady atomic.Bool + // right before the first work iteration + onStartup func(*phpThread) + // the actual work iteration (done in a loop) + onWork func(*phpThread) + // after the thread is done + onShutdown func(*phpThread) + // chan to signal the thread to stop the current work iteration + done chan struct{} + // exponential backoff for worker failures + backoff *exponentialBackoff + // known $_SERVER key names knownVariableKeys map[string]*C.zend_string } @@ -38,6 +48,7 @@ func (thread phpThread) getActiveRequest() *http.Request { return thread.mainRequest } +// TODO: Also consider this case: work => inactive => work func (thread *phpThread) setInactive() { thread.isActive.Store(false) thread.onWork = func(thread *phpThread) { @@ -65,6 +76,7 @@ func (thread *phpThread) setHooks(onStartup func(*phpThread), onWork func(*phpTh } } + // we signal to the thread to stop it's current execution and call the onStartup hook threadsReadyWG.Add(1) close(thread.done) thread.isReady.Store(false) From e52dd0fedb9875a41dcd3b28be5d575c5d0bd78a Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 14:46:57 +0100 Subject: [PATCH 18/34] Removes logs and explicitly compares to C.false. --- php_threads.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/php_threads.go b/php_threads.go index 07da30db0a..63b96c4d38 100644 --- a/php_threads.go +++ b/php_threads.go @@ -27,7 +27,6 @@ func initPHPThreads(numThreads int) error { for i := 0; i < numThreads; i++ { phpThreads[i] = &phpThread{threadIndex: i} } - logger.Warn("initializing main thread") if err := startMainThread(numThreads); err != nil { return err } @@ -36,10 +35,8 @@ func initPHPThreads(numThreads int) error { threadsReadyWG.Add(len(phpThreads)) shutdownWG.Add(len(phpThreads)) for _, thread := range phpThreads { - logger.Warn("initializing thread") thread.setInactive() - logger.Warn("thread initialized") - if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { + if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) == C.false { panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) } } From cd98e33e973a23cbd658888149b0ecd6af0df03a Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 14:49:10 +0100 Subject: [PATCH 19/34] Resets check. --- php_threads.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/php_threads.go b/php_threads.go index 63b96c4d38..6282b19f22 100644 --- a/php_threads.go +++ b/php_threads.go @@ -36,7 +36,7 @@ func initPHPThreads(numThreads int) error { shutdownWG.Add(len(phpThreads)) for _, thread := range phpThreads { thread.setInactive() - if C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) == C.false { + if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) } } From 4e2a2c61a294580c4e79b18d79feb5e19fd6ef72 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 14:52:02 +0100 Subject: [PATCH 20/34] Adds cast for safety. --- php_threads.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/php_threads.go b/php_threads.go index 6282b19f22..6233adf1cc 100644 --- a/php_threads.go +++ b/php_threads.go @@ -36,7 +36,7 @@ func initPHPThreads(numThreads int) error { shutdownWG.Add(len(phpThreads)) for _, thread := range phpThreads { thread.setInactive() - if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { + if !bool(C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex))) { panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) } } From c51eb931949484903198b4c0a6e052e97ccae8f5 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Tue, 5 Nov 2024 20:33:03 +0100 Subject: [PATCH 21/34] Fixes waitgroup overflow. --- php_thread.go | 2 +- php_threads.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/php_thread.go b/php_thread.go index 8e9232c785..259eca5874 100644 --- a/php_thread.go +++ b/php_thread.go @@ -76,7 +76,7 @@ func (thread *phpThread) setHooks(onStartup func(*phpThread), onWork func(*phpTh } } - // we signal to the thread to stop it's current execution and call the onStartup hook + // signal to the thread to stop it's current execution and call the onStartup hook threadsReadyWG.Add(1) close(thread.done) thread.isReady.Store(false) diff --git a/php_threads.go b/php_threads.go index 6233adf1cc..edc2bbfda9 100644 --- a/php_threads.go +++ b/php_threads.go @@ -21,6 +21,7 @@ var ( // reserve a fixed number of PHP threads on the go side func initPHPThreads(numThreads int) error { + threadsReadyWG = sync.WaitGroup{} threadsAreDone.Store(false) done = make(chan struct{}) phpThreads = make([]*phpThread, numThreads) @@ -36,7 +37,7 @@ func initPHPThreads(numThreads int) error { shutdownWG.Add(len(phpThreads)) for _, thread := range phpThreads { thread.setInactive() - if !bool(C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex))) { + if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) } } From 89d8e267d8df3664c7c01ff2af9fddbd3a74b9de Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Wed, 6 Nov 2024 13:45:13 +0100 Subject: [PATCH 22/34] Resolves waitgroup race condition on startup. --- frankenphp.go | 3 +-- php_thread.go | 10 ++++++---- php_threads.go | 5 +++++ php_threads_test.go | 6 +++--- worker.go | 4 ++-- 5 files changed, 17 insertions(+), 11 deletions(-) diff --git a/frankenphp.go b/frankenphp.go index ebaf955845..15870a9e40 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -335,8 +335,7 @@ func Init(options ...Option) error { } for i := 0; i < totalThreadCount-workerThreadCount; i++ { - thread := getInactivePHPThread() - thread.setHooks(nil, handleRequest, nil) + getInactivePHPThread().setActive(nil, handleRequest, nil) } if err := initWorkers(opt.workers); err != nil { diff --git a/php_thread.go b/php_thread.go index 259eca5874..183a31ca69 100644 --- a/php_thread.go +++ b/php_thread.go @@ -40,7 +40,7 @@ type phpThread struct { knownVariableKeys map[string]*C.zend_string } -func (thread phpThread) getActiveRequest() *http.Request { +func (thread *phpThread) getActiveRequest() *http.Request { if thread.workerRequest != nil { return thread.workerRequest } @@ -60,7 +60,7 @@ func (thread *phpThread) setInactive() { } } -func (thread *phpThread) setHooks(onStartup func(*phpThread), onWork func(*phpThread), onShutdown func(*phpThread)) { +func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpThread), onShutdown func(*phpThread)) { thread.isActive.Store(true) // to avoid race conditions, the thread sets its own hooks on startup @@ -77,7 +77,6 @@ func (thread *phpThread) setHooks(onStartup func(*phpThread), onWork func(*phpTh } // signal to the thread to stop it's current execution and call the onStartup hook - threadsReadyWG.Add(1) close(thread.done) thread.isReady.Store(false) } @@ -110,7 +109,10 @@ func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { if thread.onStartup != nil { thread.onStartup(thread) } - threadsReadyWG.Done() + if threadsAreBooting.Load() { + threadsReadyWG.Done() + threadsReadyWG.Wait() + } } // do the actual work diff --git a/php_threads.go b/php_threads.go index edc2bbfda9..c968c20ab5 100644 --- a/php_threads.go +++ b/php_threads.go @@ -17,6 +17,7 @@ var ( shutdownWG sync.WaitGroup done chan struct{} threadsAreDone atomic.Bool + threadsAreBooting atomic.Bool ) // reserve a fixed number of PHP threads on the go side @@ -35,6 +36,8 @@ func initPHPThreads(numThreads int) error { // initialize all threads as inactive threadsReadyWG.Add(len(phpThreads)) shutdownWG.Add(len(phpThreads)) + threadsAreBooting.Store(true) + for _, thread := range phpThreads { thread.setInactive() if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { @@ -42,6 +45,8 @@ func initPHPThreads(numThreads int) error { } } threadsReadyWG.Wait() + threadsAreBooting.Store(false) + return nil } diff --git a/php_threads_test.go b/php_threads_test.go index ea31ef2877..c8f70b6a79 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -38,7 +38,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { for i := 0; i < numThreads; i++ { newThread := getInactivePHPThread() - newThread.setHooks( + newThread.setActive( // onStartup => before the thread is ready func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { @@ -84,7 +84,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { assert.NoError(t, initPHPThreads(numThreads)) for i := 0; i < numThreads; i++ { - getInactivePHPThread().setHooks( + getInactivePHPThread().setActive( // onStartup => fake a request on startup (like a worker would do) func(thread *phpThread) { r, _ := http.NewRequest(http.MethodGet, "sleep.php", nil) @@ -136,7 +136,7 @@ func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { workWG.Add(numThreads) numberOfConversion := i for j := 0; j < numThreads; j++ { - getInactivePHPThread().setHooks( + getInactivePHPThread().setActive( // onStartup => before the thread is ready func(thread *phpThread) { startUpTypes[numberOfConversion].Add(1) diff --git a/worker.go b/worker.go index 31c8d30630..6fd2787eba 100644 --- a/worker.go +++ b/worker.go @@ -79,7 +79,7 @@ func newWorker(o workerOpt) (*worker, error) { } func (worker *worker) startNewThread() { - getInactivePHPThread().setHooks( + getInactivePHPThread().setActive( // onStartup => right before the thread is ready func(thread *phpThread) { thread.worker = worker @@ -185,7 +185,7 @@ func afterWorkerScript(thread *phpThread, exitStatus C.int) { // TODO: make the max restart configurable metrics.StopWorker(thread.worker.fileName, StopReasonRestart) - if c := logger.Check(zapcore.InfoLevel, "restarting"); c != nil { + if c := logger.Check(zapcore.DebugLevel, "restarting"); c != nil { c.Write(zap.String("worker", thread.worker.fileName)) } return From 3587243f59fe2fbd5fc4df56be80edfec7c606ce Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Thu, 7 Nov 2024 09:25:31 +0100 Subject: [PATCH 23/34] Moves worker request logic to worker.go. --- frankenphp.go | 5 +---- worker.go | 10 ++++++++-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/frankenphp.go b/frankenphp.go index 15870a9e40..67e3b667cf 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -459,10 +459,7 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error // Detect if a worker is available to handle this request if worker, ok := workers[fc.scriptFilename]; ok { - metrics.StartWorkerRequest(fc.scriptFilename) - worker.handleRequest(request) - <-fc.done - metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) + worker.handleRequest(request, fc) return nil } diff --git a/worker.go b/worker.go index 6fd2787eba..37225ddfb2 100644 --- a/worker.go +++ b/worker.go @@ -9,6 +9,7 @@ import ( "path/filepath" "sync" "sync/atomic" + "time" "github.com/dunglas/frankenphp/internal/watcher" "go.uber.org/zap" @@ -203,13 +204,17 @@ func afterWorkerScript(thread *phpThread, exitStatus C.int) { }) } -func (worker *worker) handleRequest(r *http.Request) { - worker.threadMutex.RLock() +func (worker *worker) handleRequest(r *http.Request, fc *FrankenPHPContext) { + metrics.StartWorkerRequest(fc.scriptFilename) + defer metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) + // dispatch requests to all worker threads in order + worker.threadMutex.RLock() for _, thread := range worker.threads { select { case thread.requestChan <- r: worker.threadMutex.RUnlock() + <-fc.done return default: } @@ -218,6 +223,7 @@ func (worker *worker) handleRequest(r *http.Request) { // if no thread was available, fan the request out to all threads // TODO: theoretically there could be autoscaling of threads here worker.requestChan <- r + <-fc.done } //export go_frankenphp_worker_handle_request_start From ec32f0cc55f52dc08c1b61173272d427cf54031e Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Thu, 7 Nov 2024 11:07:41 +0100 Subject: [PATCH 24/34] Removes defer. --- worker.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/worker.go b/worker.go index 37225ddfb2..1a15633240 100644 --- a/worker.go +++ b/worker.go @@ -206,7 +206,6 @@ func afterWorkerScript(thread *phpThread, exitStatus C.int) { func (worker *worker) handleRequest(r *http.Request, fc *FrankenPHPContext) { metrics.StartWorkerRequest(fc.scriptFilename) - defer metrics.StopWorkerRequest(fc.scriptFilename, time.Since(fc.startedAt)) // dispatch requests to all worker threads in order worker.threadMutex.RLock() @@ -215,15 +214,18 @@ func (worker *worker) handleRequest(r *http.Request, fc *FrankenPHPContext) { case thread.requestChan <- r: worker.threadMutex.RUnlock() <-fc.done + metrics.StopWorkerRequest(worker.fileName, time.Since(fc.startedAt)) return default: } } worker.threadMutex.RUnlock() + // if no thread was available, fan the request out to all threads // TODO: theoretically there could be autoscaling of threads here worker.requestChan <- r <-fc.done + metrics.StopWorkerRequest(worker.fileName, time.Since(fc.startedAt)) } //export go_frankenphp_worker_handle_request_start From 4e356989cd2d6597ba2b606403b54c87d4290117 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Mon, 11 Nov 2024 19:20:30 +0100 Subject: [PATCH 25/34] Removes call from go to c. --- frankenphp.c | 18 +++++++++++++----- frankenphp.go | 38 ++++++++++++++++---------------------- php_thread.go | 28 ++++++++++++++++++++++++---- php_threads_test.go | 10 +++++++--- worker.go | 7 +++++-- 5 files changed, 65 insertions(+), 36 deletions(-) diff --git a/frankenphp.c b/frankenphp.c index 7a357e0930..374607b054 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -823,7 +823,19 @@ static void *php_thread(void *arg) { should_filter_var = default_filter != NULL; // perform work until go signals to stop - while (go_frankenphp_on_thread_work(thread_index)) { + while (true) { + char *scriptName = go_frankenphp_on_thread_work(thread_index); + + // if the script name is NULL, the thread should exit + if (scriptName == NULL) { + break; + } + + // if the script name is not empty, execute the PHP script + if (strlen(scriptName) != 0) { + int exit_status = frankenphp_execute_script(scriptName); + go_frankenphp_after_thread_work(thread_index, exit_status); + } } go_frankenphp_release_known_variable_keys(thread_index); @@ -934,8 +946,6 @@ int frankenphp_request_startup() { int frankenphp_execute_script(char *file_name) { if (frankenphp_request_startup() == FAILURE) { - free(file_name); - file_name = NULL; return FAILURE; } @@ -944,8 +954,6 @@ int frankenphp_execute_script(char *file_name) { zend_file_handle file_handle; zend_stream_init_filename(&file_handle, file_name); - free(file_name); - file_name = NULL; file_handle.primary_script = 1; diff --git a/frankenphp.go b/frankenphp.go index 67e3b667cf..d7e4d2992d 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -121,7 +121,7 @@ type FrankenPHPContext struct { closed sync.Once responseWriter http.ResponseWriter - exitStatus C.int + exitStatus int done chan interface{} startedAt time.Time @@ -335,7 +335,7 @@ func Init(options ...Option) error { } for i := 0; i < totalThreadCount-workerThreadCount; i++ { - getInactivePHPThread().setActive(nil, handleRequest, nil) + getInactivePHPThread().setActive(nil, handleRequest, afterRequest, nil) } if err := initWorkers(opt.workers); err != nil { @@ -549,30 +549,33 @@ func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string func handleRequest(thread *phpThread) { select { case <-done: + thread.scriptName = "" return case r := <-requestChan: thread.mainRequest = r - - fc, ok := FromContext(r.Context()) - if !ok { - panic(InvalidRequestError) - } - defer func() { - maybeCloseContext(fc) - thread.mainRequest = nil - thread.Unpin() - }() + fc := r.Context().Value(contextKey).(*FrankenPHPContext) if err := updateServerContext(thread, r, true, false); err != nil { rejectRequest(fc.responseWriter, err.Error()) + thread.scriptName = "" + afterRequest(thread, 0) return } - fc.exitStatus = executeScriptClassic(fc.scriptFilename) + // set the scriptName that should be executed + thread.scriptName = fc.scriptFilename } } +func afterRequest(thread *phpThread, exitStatus int) { + fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) + fc.exitStatus = exitStatus + maybeCloseContext(fc) + thread.mainRequest = nil + thread.Unpin() +} + func maybeCloseContext(fc *FrankenPHPContext) { fc.closed.Do(func() { close(fc.done) @@ -783,15 +786,6 @@ func go_log(message *C.char, level C.int) { } } -func executeScriptClassic(script string) C.int { - // scriptFilename is freed in frankenphp_execute_script() - exitStatus := C.frankenphp_execute_script(C.CString(script)) - if exitStatus < 0 { - panic(ScriptExecutionError) - } - return exitStatus -} - // ExecuteScriptCLI executes the PHP script passed as parameter. // It returns the exit status code of the script. func ExecuteScriptCLI(script string, args []string) int { diff --git a/php_thread.go b/php_thread.go index 183a31ca69..d19abc31e6 100644 --- a/php_thread.go +++ b/php_thread.go @@ -20,6 +20,8 @@ type phpThread struct { requestChan chan *http.Request worker *worker + // the script name for the current request + scriptName string // the index in the phpThreads slice threadIndex int // whether the thread has work assigned to it @@ -30,6 +32,8 @@ type phpThread struct { onStartup func(*phpThread) // the actual work iteration (done in a loop) onWork func(*phpThread) + // after the work iteration is done + onWorkDone func(*phpThread, int) // after the thread is done onShutdown func(*phpThread) // chan to signal the thread to stop the current work iteration @@ -51,6 +55,7 @@ func (thread *phpThread) getActiveRequest() *http.Request { // TODO: Also consider this case: work => inactive => work func (thread *phpThread) setInactive() { thread.isActive.Store(false) + thread.scriptName = "" thread.onWork = func(thread *phpThread) { thread.requestChan = make(chan *http.Request) select { @@ -60,7 +65,7 @@ func (thread *phpThread) setInactive() { } } -func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpThread), onShutdown func(*phpThread)) { +func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpThread), onWorkDone func(*phpThread, int), onShutdown func(*phpThread)) { thread.isActive.Store(true) // to avoid race conditions, the thread sets its own hooks on startup @@ -71,6 +76,7 @@ func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpT thread.onStartup = onStartup thread.onWork = onWork thread.onShutdown = onShutdown + thread.onWorkDone = onWorkDone if thread.onStartup != nil { thread.onStartup(thread) } @@ -95,10 +101,10 @@ func (thread *phpThread) pinCString(s string) *C.char { } //export go_frankenphp_on_thread_work -func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { +func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) *C.char { // first check if FrankPHP is shutting down if threadsAreDone.Load() { - return C.bool(false) + return nil } thread := phpThreads[threadIndex] @@ -117,7 +123,21 @@ func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) C.bool { // do the actual work thread.onWork(thread) - return C.bool(true) + + // return the name of the PHP script that should be executed + return thread.pinCString(thread.scriptName) +} + +//export go_frankenphp_after_thread_work +func go_frankenphp_after_thread_work(threadIndex C.uintptr_t, exitStatus C.int) { + thread := phpThreads[threadIndex] + if exitStatus < 0 { + panic(ScriptExecutionError) + } + if thread.onWorkDone != nil { + thread.onWorkDone(thread, int(exitStatus)) + } + thread.Unpin() } //export go_frankenphp_on_thread_shutdown diff --git a/php_threads_test.go b/php_threads_test.go index c8f70b6a79..2eb251c9a5 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -53,6 +53,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { workWG.Done() newThread.setInactive() }, + nil, // onShutdown => after the thread is done func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { @@ -91,6 +92,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { r, _ = NewRequestWithContext(r, WithRequestDocumentRoot("/", false)) assert.NoError(t, updateServerContext(thread, r, true, false)) thread.mainRequest = r + thread.scriptName = scriptPath }, // onWork => execute the sleep.php script until we reach maxExecutions func(thread *phpThread) { @@ -103,9 +105,10 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { executionCount++ workWG.Done() executionMutex.Unlock() - - // exit the loop and fail the test if the script fails - if int(executeScriptClassic(scriptPath)) != 0 { + }, + // onWorkDone => check the exit status of the script + func(thread *phpThread, existStatus int) { + if int(existStatus) != 0 { panic("script execution failed: " + scriptPath) } }, @@ -147,6 +150,7 @@ func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { thread.setInactive() workWG.Done() }, + nil, // onShutdown => after the thread is done func(thread *phpThread) { shutdownTypes[numberOfConversion].Add(1) diff --git a/worker.go b/worker.go index 1a15633240..3f50c01130 100644 --- a/worker.go +++ b/worker.go @@ -90,6 +90,7 @@ func (worker *worker) startNewThread() { worker.threadMutex.Lock() worker.threads = append(worker.threads, thread) worker.threadMutex.Unlock() + thread.scriptName = worker.fileName }, // onWork => while the thread is working (in a loop) func(thread *phpThread) { @@ -98,7 +99,9 @@ func (worker *worker) startNewThread() { workerRestartWG.Wait() } beforeWorkerScript(thread) - exitStatus := executeScriptClassic(thread.worker.fileName) + }, + // onWorkDone => after the work iteration is done + func(thread *phpThread, exitStatus int) { afterWorkerScript(thread, exitStatus) }, // onShutdown => after the thread is done @@ -171,7 +174,7 @@ func beforeWorkerScript(thread *phpThread) { } } -func afterWorkerScript(thread *phpThread, exitStatus C.int) { +func afterWorkerScript(thread *phpThread, exitStatus int) { fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) fc.exitStatus = exitStatus From 8a272cba7c382ffb204e75c6f765eba267285208 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Fri, 15 Nov 2024 12:58:06 +0100 Subject: [PATCH 26/34] Fixes merge conflict. --- frankenphp.go | 3 +-- php_threads_test.go | 6 +++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/frankenphp.go b/frankenphp.go index a61b826a7f..b5d2bca487 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -558,8 +558,7 @@ func go_sapi_getenv(threadIndex C.uintptr_t, name *C.go_string) *C.char { return phpThreads[threadIndex].pinCString(envValue) } -//export go_handle_request -func go_handle_request(threadIndex C.uintptr_t) bool { +func handleRequest(thread *phpThread) { select { case <-done: thread.scriptName = "" diff --git a/php_threads_test.go b/php_threads_test.go index 2eb251c9a5..51228a6956 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -12,8 +12,8 @@ import ( ) func TestStartAndStopTheMainThreadWithOneInactiveThread(t *testing.T) { - logger = zap.NewNop() // the logger needs to not be nil - assert.NoError(t, initPHPThreads(1)) // reserve 1 thread + logger = zap.NewNop() // the logger needs to not be nil + assert.NoError(t, initPHPThreads(1)) // reserve 1 thread assert.Len(t, phpThreads, 1) assert.Equal(t, 0, phpThreads[0].threadIndex) @@ -53,7 +53,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { workWG.Done() newThread.setInactive() }, - nil, + nil, // onShutdown => after the thread is done func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { From ecce5d52b45b50994085abd73dfc9d9de56daf56 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Fri, 15 Nov 2024 13:00:48 +0100 Subject: [PATCH 27/34] Adds fibers test back in. --- frankenphp_test.go | 17 +++++++++++++++++ testdata/fiber-basic.php | 9 +++++++++ 2 files changed, 26 insertions(+) create mode 100644 testdata/fiber-basic.php diff --git a/frankenphp_test.go b/frankenphp_test.go index 9ca6b1520b..436b96b19e 100644 --- a/frankenphp_test.go +++ b/frankenphp_test.go @@ -592,6 +592,23 @@ func testFiberNoCgo(t *testing.T, opts *testOptions) { }, opts) } +func TestFiberBasic_module(t *testing.T) { testFiberBasic(t, &testOptions{}) } +func TestFiberBasic_worker(t *testing.T) { + testFiberBasic(t, &testOptions{workerScript: "fiber-basic.php"}) +} +func testFiberBasic(t *testing.T, opts *testOptions) { + runTest(t, func(handler func(http.ResponseWriter, *http.Request), _ *httptest.Server, i int) { + req := httptest.NewRequest("GET", fmt.Sprintf("http://example.com/fiber-basic.php?i=%d", i), nil) + w := httptest.NewRecorder() + handler(w, req) + + resp := w.Result() + body, _ := io.ReadAll(resp.Body) + + assert.Equal(t, string(body), fmt.Sprintf("Fiber %d", i)) + }, opts) +} + func TestRequestHeaders_module(t *testing.T) { testRequestHeaders(t, &testOptions{}) } func TestRequestHeaders_worker(t *testing.T) { testRequestHeaders(t, &testOptions{workerScript: "request-headers.php"}) diff --git a/testdata/fiber-basic.php b/testdata/fiber-basic.php new file mode 100644 index 0000000000..bdb52336f6 --- /dev/null +++ b/testdata/fiber-basic.php @@ -0,0 +1,9 @@ +start(); +}; From 06ebd67cf4b7519db9537775926b9172c608d6ed Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Fri, 15 Nov 2024 19:39:30 +0100 Subject: [PATCH 28/34] Refactors new thread loop approach. --- env.go | 84 +++++++++++++++++++++++++++++++++++++++++++ frankenphp.c | 4 +-- frankenphp.go | 88 +++------------------------------------------ php_thread.go | 38 ++++++++++---------- php_threads.go | 1 - php_threads_test.go | 18 ++++++---- worker.go | 87 ++++++++++++++++++++++---------------------- 7 files changed, 163 insertions(+), 157 deletions(-) create mode 100644 env.go diff --git a/env.go b/env.go new file mode 100644 index 0000000000..f95c6fd138 --- /dev/null +++ b/env.go @@ -0,0 +1,84 @@ +package frankenphp + +// #include "frankenphp.h" +import "C" +import ( + "os" + "strings" + "unsafe" +) + +//export go_putenv +func go_putenv(str *C.char, length C.int) C.bool { + // Create a byte slice from C string with a specified length + s := C.GoBytes(unsafe.Pointer(str), length) + + // Convert byte slice to string + envString := string(s) + + // Check if '=' is present in the string + if key, val, found := strings.Cut(envString, "="); found { + if os.Setenv(key, val) != nil { + return false // Failure + } + } else { + // No '=', unset the environment variable + if os.Unsetenv(envString) != nil { + return false // Failure + } + } + + return true // Success +} + +//export go_getfullenv +func go_getfullenv(threadIndex C.uintptr_t) (*C.go_string, C.size_t) { + thread := phpThreads[threadIndex] + + env := os.Environ() + goStrings := make([]C.go_string, len(env)*2) + + for i, envVar := range env { + key, val, _ := strings.Cut(envVar, "=") + goStrings[i*2] = C.go_string{C.size_t(len(key)), thread.pinString(key)} + goStrings[i*2+1] = C.go_string{C.size_t(len(val)), thread.pinString(val)} + } + + value := unsafe.SliceData(goStrings) + thread.Pin(value) + + return value, C.size_t(len(env)) +} + +//export go_getenv +func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string) { + thread := phpThreads[threadIndex] + + // Create a byte slice from C string with a specified length + envName := C.GoStringN(name.data, C.int(name.len)) + + // Get the environment variable value + envValue, exists := os.LookupEnv(envName) + if !exists { + // Environment variable does not exist + return false, nil // Return 0 to indicate failure + } + + // Convert Go string to C string + value := &C.go_string{C.size_t(len(envValue)), thread.pinString(envValue)} + thread.Pin(value) + + return true, value // Return 1 to indicate success +} + +//export go_sapi_getenv +func go_sapi_getenv(threadIndex C.uintptr_t, name *C.go_string) *C.char { + envName := C.GoStringN(name.data, C.int(name.len)) + + envValue, exists := os.LookupEnv(envName) + if !exists { + return nil + } + + return phpThreads[threadIndex].pinCString(envValue) +} diff --git a/frankenphp.c b/frankenphp.c index 8492dcda0e..0b249e1529 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -830,7 +830,7 @@ static void *php_thread(void *arg) { // perform work until go signals to stop while (true) { - char *scriptName = go_frankenphp_on_thread_work(thread_index); + char *scriptName = go_frankenphp_before_script_execution(thread_index); // if the script name is NULL, the thread should exit if (scriptName == NULL) { @@ -840,7 +840,7 @@ static void *php_thread(void *arg) { // if the script name is not empty, execute the PHP script if (strlen(scriptName) != 0) { int exit_status = frankenphp_execute_script(scriptName); - go_frankenphp_after_thread_work(thread_index, exit_status); + go_frankenphp_after_script_execution(thread_index, exit_status); } } diff --git a/frankenphp.go b/frankenphp.go index b5d2bca487..7b7f61b6ae 100644 --- a/frankenphp.go +++ b/frankenphp.go @@ -476,91 +476,10 @@ func ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) error return nil } -//export go_putenv -func go_putenv(str *C.char, length C.int) C.bool { - // Create a byte slice from C string with a specified length - s := C.GoBytes(unsafe.Pointer(str), length) - - // Convert byte slice to string - envString := string(s) - - // Check if '=' is present in the string - if key, val, found := strings.Cut(envString, "="); found { - if os.Setenv(key, val) != nil { - return false // Failure - } - } else { - // No '=', unset the environment variable - if os.Unsetenv(envString) != nil { - return false // Failure - } - } - - return true // Success -} - -//export go_getfullenv -func go_getfullenv(threadIndex C.uintptr_t) (*C.go_string, C.size_t) { - thread := phpThreads[threadIndex] - - env := os.Environ() - goStrings := make([]C.go_string, len(env)*2) - - for i, envVar := range env { - key, val, _ := strings.Cut(envVar, "=") - k := unsafe.StringData(key) - v := unsafe.StringData(val) - thread.Pin(k) - thread.Pin(v) - - goStrings[i*2] = C.go_string{C.size_t(len(key)), (*C.char)(unsafe.Pointer(k))} - goStrings[i*2+1] = C.go_string{C.size_t(len(val)), (*C.char)(unsafe.Pointer(v))} - } - - value := unsafe.SliceData(goStrings) - thread.Pin(value) - - return value, C.size_t(len(env)) -} - -//export go_getenv -func go_getenv(threadIndex C.uintptr_t, name *C.go_string) (C.bool, *C.go_string) { - thread := phpThreads[threadIndex] - - // Create a byte slice from C string with a specified length - envName := C.GoStringN(name.data, C.int(name.len)) - - // Get the environment variable value - envValue, exists := os.LookupEnv(envName) - if !exists { - // Environment variable does not exist - return false, nil // Return 0 to indicate failure - } - - // Convert Go string to C string - val := unsafe.StringData(envValue) - thread.Pin(val) - value := &C.go_string{C.size_t(len(envValue)), (*C.char)(unsafe.Pointer(val))} - thread.Pin(value) - - return true, value // Return 1 to indicate success -} - -//export go_sapi_getenv -func go_sapi_getenv(threadIndex C.uintptr_t, name *C.go_string) *C.char { - envName := C.GoStringN(name.data, C.int(name.len)) - - envValue, exists := os.LookupEnv(envName) - if !exists { - return nil - } - - return phpThreads[threadIndex].pinCString(envValue) -} - func handleRequest(thread *phpThread) { select { case <-done: + // no script should be executed if the server is shutting down thread.scriptName = "" return @@ -570,8 +489,10 @@ func handleRequest(thread *phpThread) { if err := updateServerContext(thread, r, true, false); err != nil { rejectRequest(fc.responseWriter, err.Error()) - thread.scriptName = "" afterRequest(thread, 0) + thread.Unpin() + // no script should be executed if the request was rejected + thread.scriptName = "" return } @@ -585,7 +506,6 @@ func afterRequest(thread *phpThread, exitStatus int) { fc.exitStatus = exitStatus maybeCloseContext(fc) thread.mainRequest = nil - thread.Unpin() } func maybeCloseContext(fc *FrankenPHPContext) { diff --git a/php_thread.go b/php_thread.go index d19abc31e6..5c00959b08 100644 --- a/php_thread.go +++ b/php_thread.go @@ -1,8 +1,5 @@ package frankenphp -// #include -// #include -// #include // #include "frankenphp.h" import "C" import ( @@ -31,9 +28,9 @@ type phpThread struct { // right before the first work iteration onStartup func(*phpThread) // the actual work iteration (done in a loop) - onWork func(*phpThread) + beforeScriptExecution func(*phpThread) // after the work iteration is done - onWorkDone func(*phpThread, int) + afterScriptExecution func(*phpThread, int) // after the thread is done onShutdown func(*phpThread) // chan to signal the thread to stop the current work iteration @@ -56,7 +53,7 @@ func (thread *phpThread) getActiveRequest() *http.Request { func (thread *phpThread) setInactive() { thread.isActive.Store(false) thread.scriptName = "" - thread.onWork = func(thread *phpThread) { + thread.beforeScriptExecution = func(thread *phpThread) { thread.requestChan = make(chan *http.Request) select { case <-done: @@ -65,7 +62,12 @@ func (thread *phpThread) setInactive() { } } -func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpThread), onWorkDone func(*phpThread, int), onShutdown func(*phpThread)) { +func (thread *phpThread) setActive( + onStartup func(*phpThread), + beforeScriptExecution func(*phpThread), + afterScriptExecution func(*phpThread, int), + onShutdown func(*phpThread), +) { thread.isActive.Store(true) // to avoid race conditions, the thread sets its own hooks on startup @@ -74,9 +76,9 @@ func (thread *phpThread) setActive(onStartup func(*phpThread), onWork func(*phpT thread.onShutdown(thread) } thread.onStartup = onStartup - thread.onWork = onWork + thread.beforeScriptExecution = beforeScriptExecution thread.onShutdown = onShutdown - thread.onWorkDone = onWorkDone + thread.afterScriptExecution = afterScriptExecution if thread.onStartup != nil { thread.onStartup(thread) } @@ -100,9 +102,9 @@ func (thread *phpThread) pinCString(s string) *C.char { return thread.pinString(s + "\x00") } -//export go_frankenphp_on_thread_work -func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) *C.char { - // first check if FrankPHP is shutting down +//export go_frankenphp_before_script_execution +func go_frankenphp_before_script_execution(threadIndex C.uintptr_t) *C.char { + // returning nil signals the thread to stop if threadsAreDone.Load() { return nil } @@ -121,21 +123,21 @@ func go_frankenphp_on_thread_work(threadIndex C.uintptr_t) *C.char { } } - // do the actual work - thread.onWork(thread) + // execute a hook before the script is executed + thread.beforeScriptExecution(thread) // return the name of the PHP script that should be executed return thread.pinCString(thread.scriptName) } -//export go_frankenphp_after_thread_work -func go_frankenphp_after_thread_work(threadIndex C.uintptr_t, exitStatus C.int) { +//export go_frankenphp_after_script_execution +func go_frankenphp_after_script_execution(threadIndex C.uintptr_t, exitStatus C.int) { thread := phpThreads[threadIndex] if exitStatus < 0 { panic(ScriptExecutionError) } - if thread.onWorkDone != nil { - thread.onWorkDone(thread, int(exitStatus)) + if thread.afterScriptExecution != nil { + thread.afterScriptExecution(thread, int(exitStatus)) } thread.Unpin() } diff --git a/php_threads.go b/php_threads.go index c968c20ab5..11826ba5ac 100644 --- a/php_threads.go +++ b/php_threads.go @@ -1,6 +1,5 @@ package frankenphp -// #include // #include "frankenphp.h" import "C" import ( diff --git a/php_threads_test.go b/php_threads_test.go index 51228a6956..b290e0c779 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -45,7 +45,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { readyThreads.Add(1) } }, - // onWork => while the thread is running (we stop here immediately) + // beforeScriptExecution => we stop here immediately func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { workingThreads.Add(1) @@ -53,7 +53,10 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { workWG.Done() newThread.setInactive() }, - nil, + // afterScriptExecution => no script is executed, we shouldn't reach here + func(thread *phpThread, exitStatus int) { + panic("hook afterScriptExecution should not be called here") + }, // onShutdown => after the thread is done func(thread *phpThread) { if thread.threadIndex == newThread.threadIndex { @@ -94,7 +97,7 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { thread.mainRequest = r thread.scriptName = scriptPath }, - // onWork => execute the sleep.php script until we reach maxExecutions + // beforeScriptExecution => execute the sleep.php script until we reach maxExecutions func(thread *phpThread) { executionMutex.Lock() if executionCount >= maxExecutions { @@ -106,9 +109,9 @@ func TestSleep10000TimesIn100Threads(t *testing.T) { workWG.Done() executionMutex.Unlock() }, - // onWorkDone => check the exit status of the script - func(thread *phpThread, existStatus int) { - if int(existStatus) != 0 { + // afterScriptExecution => check the exit status of the script + func(thread *phpThread, exitStatus int) { + if int(exitStatus) != 0 { panic("script execution failed: " + scriptPath) } }, @@ -144,12 +147,13 @@ func TestStart100ThreadsAndConvertThemToDifferentThreads10Times(t *testing.T) { func(thread *phpThread) { startUpTypes[numberOfConversion].Add(1) }, - // onWork => while the thread is running + // beforeScriptExecution => while the thread is running func(thread *phpThread) { workTypes[numberOfConversion].Add(1) thread.setInactive() workWG.Done() }, + // afterScriptExecution => we don't execute a script nil, // onShutdown => after the thread is done func(thread *phpThread) { diff --git a/worker.go b/worker.go index 53e03c85d7..01ef153aa8 100644 --- a/worker.go +++ b/worker.go @@ -1,6 +1,5 @@ package frankenphp -// #include // #include "frankenphp.h" import "C" import ( @@ -80,39 +79,6 @@ func newWorker(o workerOpt) (*worker, error) { return w, nil } -func (worker *worker) startNewThread() { - getInactivePHPThread().setActive( - // onStartup => right before the thread is ready - func(thread *phpThread) { - thread.worker = worker - thread.requestChan = make(chan *http.Request) - metrics.ReadyWorker(worker.fileName) - thread.backoff = newExponentialBackoff() - worker.threadMutex.Lock() - worker.threads = append(worker.threads, thread) - worker.threadMutex.Unlock() - thread.scriptName = worker.fileName - }, - // onWork => while the thread is working (in a loop) - func(thread *phpThread) { - if watcherIsEnabled && workersAreRestarting.Load() { - workerShutdownWG.Done() - workerRestartWG.Wait() - } - beforeWorkerScript(thread) - }, - // onWorkDone => after the work iteration is done - func(thread *phpThread, exitStatus int) { - afterWorkerScript(thread, exitStatus) - }, - // onShutdown => after the thread is done - func(thread *phpThread) { - thread.worker = nil - thread.backoff = nil - }, - ) -} - func stopWorkers() { close(workersDone) } @@ -129,7 +95,7 @@ func restartWorkers() { workerShutdownWG.Add(worker.num) } workersAreRestarting.Store(true) - close(workersDone) + stopWorkers() workerShutdownWG.Wait() workersDone = make(chan interface{}) workersAreRestarting.Store(false) @@ -143,10 +109,42 @@ func getDirectoriesToWatch(workerOpts []workerOpt) []string { return directoriesToWatch } -func beforeWorkerScript(thread *phpThread) { - worker := thread.worker +func (worker *worker) startNewThread() { + getInactivePHPThread().setActive( + // onStartup => right before the thread is ready + func(thread *phpThread) { + thread.worker = worker + thread.scriptName = worker.fileName + thread.requestChan = make(chan *http.Request) + thread.backoff = newExponentialBackoff() + worker.threadMutex.Lock() + worker.threads = append(worker.threads, thread) + worker.threadMutex.Unlock() + metrics.ReadyWorker(worker.fileName) + }, + // beforeScriptExecution => set up the worker with a fake request + func(thread *phpThread) { + worker.beforeScript(thread) + }, + // afterScriptExecution => tear down the worker + func(thread *phpThread, exitStatus int) { + worker.afterScript(thread, exitStatus) + }, + // onShutdown => after the thread is done + func(thread *phpThread) { + thread.worker = nil + thread.backoff = nil + }, + ) +} + +func (worker *worker) beforeScript(thread *phpThread) { + // if we are restarting due to file watching, wait for all workers to finish first + if watcherIsEnabled && workersAreRestarting.Load() { + workerShutdownWG.Done() + workerRestartWG.Wait() + } - // if we are restarting the worker, reset the exponential failure backoff thread.backoff.reset() metrics.StartWorker(worker.fileName) @@ -175,36 +173,35 @@ func beforeWorkerScript(thread *phpThread) { } } -func afterWorkerScript(thread *phpThread, exitStatus int) { +func (worker *worker) afterScript(thread *phpThread, exitStatus int) { fc := thread.mainRequest.Context().Value(contextKey).(*FrankenPHPContext) fc.exitStatus = exitStatus defer func() { maybeCloseContext(fc) thread.mainRequest = nil - thread.Unpin() }() // on exit status 0 we just run the worker script again if fc.exitStatus == 0 { // TODO: make the max restart configurable - metrics.StopWorker(thread.worker.fileName, StopReasonRestart) + metrics.StopWorker(worker.fileName, StopReasonRestart) if c := logger.Check(zapcore.DebugLevel, "restarting"); c != nil { - c.Write(zap.String("worker", thread.worker.fileName)) + c.Write(zap.String("worker", worker.fileName)) } return } // on exit status 1 we apply an exponential backoff when restarting - metrics.StopWorker(thread.worker.fileName, StopReasonCrash) + metrics.StopWorker(worker.fileName, StopReasonCrash) thread.backoff.trigger(func(failureCount int) { // if we end up here, the worker has not been up for backoff*2 // this is probably due to a syntax error or another fatal error if !watcherIsEnabled { - panic(fmt.Errorf("workers %q: too many consecutive failures", thread.worker.fileName)) + panic(fmt.Errorf("workers %q: too many consecutive failures", worker.fileName)) } - logger.Warn("many consecutive worker failures", zap.String("worker", thread.worker.fileName), zap.Int("failures", failureCount)) + logger.Warn("many consecutive worker failures", zap.String("worker", worker.fileName), zap.Int("failures", failureCount)) }) } From c811f4a167cde72eed5651cfc7ed3cfea859b262 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 16 Nov 2024 16:57:45 +0100 Subject: [PATCH 29/34] Removes redundant check. --- worker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker.go b/worker.go index 01ef153aa8..1b245b2dac 100644 --- a/worker.go +++ b/worker.go @@ -140,7 +140,7 @@ func (worker *worker) startNewThread() { func (worker *worker) beforeScript(thread *phpThread) { // if we are restarting due to file watching, wait for all workers to finish first - if watcherIsEnabled && workersAreRestarting.Load() { + if workersAreRestarting.Load() { workerShutdownWG.Done() workerRestartWG.Wait() } From 6bd047a4cc6b69c1acfa45b819786e0706259d96 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sat, 16 Nov 2024 16:58:00 +0100 Subject: [PATCH 30/34] Adds compareAndSwap. --- php_thread.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/php_thread.go b/php_thread.go index 5c00959b08..a8d64ce413 100644 --- a/php_thread.go +++ b/php_thread.go @@ -111,8 +111,7 @@ func go_frankenphp_before_script_execution(threadIndex C.uintptr_t) *C.char { thread := phpThreads[threadIndex] // if the thread is not ready, set it up - if !thread.isReady.Load() { - thread.isReady.Store(true) + if thread.isReady.CompareAndSwap(false, true) { thread.done = make(chan struct{}) if thread.onStartup != nil { thread.onStartup(thread) From 55ad8ba8bcde8937374ee849302a291fcda21220 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sun, 17 Nov 2024 22:39:57 +0100 Subject: [PATCH 31/34] Refactor: removes global waitgroups and uses a 'thread state' abstraction instead. --- frankenphp.c | 2 + php_thread.go | 47 ++++++++++---------- php_threads.go | 71 +++++++++++++++-------------- php_threads_test.go | 4 +- thread_state.go | 103 +++++++++++++++++++++++++++++++++++++++++++ thread_state_test.go | 43 ++++++++++++++++++ worker.go | 40 +++++++++-------- 7 files changed, 233 insertions(+), 77 deletions(-) create mode 100644 thread_state.go create mode 100644 thread_state_test.go diff --git a/frankenphp.c b/frankenphp.c index 0b249e1529..73a0dc0bed 100644 --- a/frankenphp.c +++ b/frankenphp.c @@ -828,6 +828,8 @@ static void *php_thread(void *arg) { cfg_get_string("filter.default", &default_filter); should_filter_var = default_filter != NULL; + go_frankenphp_on_thread_startup(thread_index); + // perform work until go signals to stop while (true) { char *scriptName = go_frankenphp_before_script_execution(thread_index); diff --git a/php_thread.go b/php_thread.go index a8d64ce413..bd260f6c70 100644 --- a/php_thread.go +++ b/php_thread.go @@ -39,6 +39,8 @@ type phpThread struct { backoff *exponentialBackoff // known $_SERVER key names knownVariableKeys map[string]*C.zend_string + // the state handler + state *threadStateHandler } func (thread *phpThread) getActiveRequest() *http.Request { @@ -49,16 +51,11 @@ func (thread *phpThread) getActiveRequest() *http.Request { return thread.mainRequest } -// TODO: Also consider this case: work => inactive => work func (thread *phpThread) setInactive() { - thread.isActive.Store(false) thread.scriptName = "" - thread.beforeScriptExecution = func(thread *phpThread) { - thread.requestChan = make(chan *http.Request) - select { - case <-done: - case <-thread.done: - } + // TODO: handle this in a state machine + if !thread.state.is(stateShuttingDown) { + thread.state.set(stateInactive) } } @@ -68,8 +65,6 @@ func (thread *phpThread) setActive( afterScriptExecution func(*phpThread, int), onShutdown func(*phpThread), ) { - thread.isActive.Store(true) - // to avoid race conditions, the thread sets its own hooks on startup thread.onStartup = func(thread *phpThread) { if thread.onShutdown != nil { @@ -83,10 +78,7 @@ func (thread *phpThread) setActive( thread.onStartup(thread) } } - - // signal to the thread to stop it's current execution and call the onStartup hook - close(thread.done) - thread.isReady.Store(false) + thread.state.set(stateActive) } // Pin a string that is not null-terminated @@ -102,24 +94,31 @@ func (thread *phpThread) pinCString(s string) *C.char { return thread.pinString(s + "\x00") } +//export go_frankenphp_on_thread_startup +func go_frankenphp_on_thread_startup(threadIndex C.uintptr_t) { + phpThreads[threadIndex].setInactive() +} + //export go_frankenphp_before_script_execution func go_frankenphp_before_script_execution(threadIndex C.uintptr_t) *C.char { + thread := phpThreads[threadIndex] + + // if the state is inactive, wait for it to be active + if thread.state.is(stateInactive) { + thread.state.waitFor(stateActive, stateShuttingDown) + } + // returning nil signals the thread to stop - if threadsAreDone.Load() { + if thread.state.is(stateShuttingDown) { return nil } - thread := phpThreads[threadIndex] - // if the thread is not ready, set it up - if thread.isReady.CompareAndSwap(false, true) { - thread.done = make(chan struct{}) + // if the thread is not ready yet, set it up + if !thread.state.is(stateReady) { + thread.state.set(stateReady) if thread.onStartup != nil { thread.onStartup(thread) } - if threadsAreBooting.Load() { - threadsReadyWG.Done() - threadsReadyWG.Wait() - } } // execute a hook before the script is executed @@ -148,5 +147,5 @@ func go_frankenphp_on_thread_shutdown(threadIndex C.uintptr_t) { if thread.onShutdown != nil { thread.onShutdown(thread) } - shutdownWG.Done() + thread.state.set(stateDone) } diff --git a/php_threads.go b/php_threads.go index 11826ba5ac..9ef71fde81 100644 --- a/php_threads.go +++ b/php_threads.go @@ -5,73 +5,78 @@ import "C" import ( "fmt" "sync" - "sync/atomic" ) var ( - phpThreads []*phpThread - terminationWG sync.WaitGroup - mainThreadShutdownWG sync.WaitGroup - threadsReadyWG sync.WaitGroup - shutdownWG sync.WaitGroup - done chan struct{} - threadsAreDone atomic.Bool - threadsAreBooting atomic.Bool + phpThreads []*phpThread + done chan struct{} + mainThreadState *threadStateHandler ) // reserve a fixed number of PHP threads on the go side func initPHPThreads(numThreads int) error { - threadsReadyWG = sync.WaitGroup{} - threadsAreDone.Store(false) done = make(chan struct{}) phpThreads = make([]*phpThread, numThreads) for i := 0; i < numThreads; i++ { - phpThreads[i] = &phpThread{threadIndex: i} + phpThreads[i] = &phpThread{ + threadIndex: i, + state: &threadStateHandler{currentState: stateBooting}, + } } if err := startMainThread(numThreads); err != nil { return err } // initialize all threads as inactive - threadsReadyWG.Add(len(phpThreads)) - shutdownWG.Add(len(phpThreads)) - threadsAreBooting.Store(true) + ready := sync.WaitGroup{} + ready.Add(len(phpThreads)) for _, thread := range phpThreads { - thread.setInactive() - if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { - panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) - } + go func() { + if !C.frankenphp_new_php_thread(C.uintptr_t(thread.threadIndex)) { + panic(fmt.Sprintf("unable to create thread %d", thread.threadIndex)) + } + thread.state.waitFor(stateInactive) + ready.Done() + }() } - threadsReadyWG.Wait() - threadsAreBooting.Store(false) + + ready.Wait() return nil } func drainPHPThreads() { - threadsAreDone.Store(true) + doneWG := sync.WaitGroup{} + doneWG.Add(len(phpThreads)) + for _, thread := range phpThreads { + thread.state.set(stateShuttingDown) + } close(done) - shutdownWG.Wait() - mainThreadShutdownWG.Done() - terminationWG.Wait() + for _, thread := range phpThreads { + go func(thread *phpThread) { + thread.state.waitFor(stateDone) + doneWG.Done() + }(thread) + } + doneWG.Wait() + mainThreadState.set(stateShuttingDown) + mainThreadState.waitFor(stateDone) phpThreads = nil } func startMainThread(numThreads int) error { - threadsReadyWG.Add(1) - mainThreadShutdownWG.Add(1) - terminationWG.Add(1) + mainThreadState = &threadStateHandler{currentState: stateBooting} if C.frankenphp_new_main_thread(C.int(numThreads)) != 0 { return MainThreadCreationError } - threadsReadyWG.Wait() + mainThreadState.waitFor(stateActive) return nil } func getInactivePHPThread() *phpThread { for _, thread := range phpThreads { - if !thread.isActive.Load() { + if thread.state.is(stateInactive) { return thread } } @@ -80,11 +85,11 @@ func getInactivePHPThread() *phpThread { //export go_frankenphp_main_thread_is_ready func go_frankenphp_main_thread_is_ready() { - threadsReadyWG.Done() - mainThreadShutdownWG.Wait() + mainThreadState.set(stateActive) + mainThreadState.waitFor(stateShuttingDown) } //export go_frankenphp_shutdown_main_thread func go_frankenphp_shutdown_main_thread() { - terminationWG.Done() + mainThreadState.set(stateDone) } diff --git a/php_threads_test.go b/php_threads_test.go index b290e0c779..ab85c783fe 100644 --- a/php_threads_test.go +++ b/php_threads_test.go @@ -17,7 +17,7 @@ func TestStartAndStopTheMainThreadWithOneInactiveThread(t *testing.T) { assert.Len(t, phpThreads, 1) assert.Equal(t, 0, phpThreads[0].threadIndex) - assert.False(t, phpThreads[0].isActive.Load()) + assert.True(t, phpThreads[0].state.is(stateInactive)) assert.Nil(t, phpThreads[0].worker) drainPHPThreads() @@ -76,7 +76,7 @@ func TestStartAndStop100PHPThreadsThatDoNothing(t *testing.T) { // This test calls sleep() 10.000 times for 1ms in 100 PHP threads. func TestSleep10000TimesIn100Threads(t *testing.T) { - logger = zap.NewNop() // the logger needs to not be nil + logger, _ = zap.NewDevelopment() // the logger needs to not be nil numThreads := 100 maxExecutions := 10000 executionMutex := sync.Mutex{} diff --git a/thread_state.go b/thread_state.go new file mode 100644 index 0000000000..00540610b3 --- /dev/null +++ b/thread_state.go @@ -0,0 +1,103 @@ +package frankenphp + +import ( + "slices" + "sync" +) + +type threadState int + +const ( + stateBooting threadState = iota + stateInactive + stateActive + stateReady + stateWorking + stateShuttingDown + stateDone + stateRestarting +) + +type threadStateHandler struct { + currentState threadState + mu sync.RWMutex + subscribers []stateSubscriber +} + +type stateSubscriber struct { + states []threadState + ch chan struct{} + yieldFor *sync.WaitGroup +} + +func (h *threadStateHandler) is(state threadState) bool { + h.mu.RLock() + defer h.mu.RUnlock() + return h.currentState == state +} + +func (h *threadStateHandler) get() threadState { + h.mu.RLock() + defer h.mu.RUnlock() + return h.currentState +} + +func (h *threadStateHandler) set(nextState threadState) { + h.mu.Lock() + defer h.mu.Unlock() + if h.currentState == nextState { + // TODO: do we return here or inform subscribers? + // TODO: should we ever reach here? + return + } + + h.currentState = nextState + + if len(h.subscribers) == 0 { + return + } + + newSubscribers := []stateSubscriber{} + // TODO: do we even need multiple subscribers? + // notify subscribers to the state change + for _, sub := range h.subscribers { + if !slices.Contains(sub.states, nextState) { + newSubscribers = append(newSubscribers, sub) + continue + } + close(sub.ch) + // yield for the subscriber + if sub.yieldFor != nil { + defer sub.yieldFor.Wait() + } + } + h.subscribers = newSubscribers +} + +// wait for the thread to reach a certain state +func (h *threadStateHandler) waitFor(states ...threadState) { + h.waitForStates(states, nil) +} + +// make the thread yield to a WaitGroup once it reaches the state +// this makes sure all threads are in sync both ways +func (h *threadStateHandler) waitForAndYield(yieldFor *sync.WaitGroup, states ...threadState) { + h.waitForStates(states, yieldFor) +} + +// subscribe to a state and wait until the thread reaches it +func (h *threadStateHandler) waitForStates(states []threadState, yieldFor *sync.WaitGroup) { + h.mu.Lock() + if slices.Contains(states, h.currentState) { + h.mu.Unlock() + return + } + sub := stateSubscriber{ + states: states, + ch: make(chan struct{}), + yieldFor: yieldFor, + } + h.subscribers = append(h.subscribers, sub) + h.mu.Unlock() + <-sub.ch +} diff --git a/thread_state_test.go b/thread_state_test.go new file mode 100644 index 0000000000..10d42635a8 --- /dev/null +++ b/thread_state_test.go @@ -0,0 +1,43 @@ +package frankenphp + +import ( + "sync" + "testing" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap" +) + +func TestYieldToEachOtherViaThreadStates(t *testing.T) { + threadState := &threadStateHandler{currentState: stateBooting} + + go func() { + threadState.waitFor(stateInactive) + assert.True(t, threadState.is(stateInactive)) + threadState.set(stateActive) + }() + + threadState.set(stateInactive) + threadState.waitFor(stateActive) + assert.True(t, threadState.is(stateActive)) +} + +func TestYieldToAWaitGroupPassedByThreadState(t *testing.T) { + logger, _ = zap.NewDevelopment() + threadState := &threadStateHandler{currentState: stateBooting} + hasYielded := false + wg := sync.WaitGroup{} + wg.Add(1) + + go func() { + threadState.set(stateInactive) + threadState.waitForAndYield(&wg, stateActive) + hasYielded = true + wg.Done() + }() + + threadState.waitFor(stateInactive) + threadState.set(stateActive) + // the state should be 'ready' since we are also yielding to the WaitGroup + assert.True(t, hasYielded) +} diff --git a/worker.go b/worker.go index 1b245b2dac..60722867ca 100644 --- a/worker.go +++ b/worker.go @@ -8,7 +8,6 @@ import ( "net/http" "path/filepath" "sync" - "sync/atomic" "time" "github.com/dunglas/frankenphp/internal/watcher" @@ -26,12 +25,9 @@ type worker struct { } var ( - workers map[string]*worker - workersDone chan interface{} - watcherIsEnabled bool - workersAreRestarting atomic.Bool - workerRestartWG sync.WaitGroup - workerShutdownWG sync.WaitGroup + workers map[string]*worker + workersDone chan interface{} + watcherIsEnabled bool ) func initWorkers(opt []workerOpt) error { @@ -89,16 +85,25 @@ func drainWorkers() { } func restartWorkers() { - workerRestartWG.Add(1) - defer workerRestartWG.Done() + restart := sync.WaitGroup{} + restart.Add(1) + ready := sync.WaitGroup{} for _, worker := range workers { - workerShutdownWG.Add(worker.num) + worker.threadMutex.RLock() + ready.Add(len(worker.threads)) + for _, thread := range worker.threads { + thread.state.set(stateRestarting) + go func(thread *phpThread) { + thread.state.waitForAndYield(&restart, stateReady) + ready.Done() + }(thread) + } + worker.threadMutex.RUnlock() } - workersAreRestarting.Store(true) stopWorkers() - workerShutdownWG.Wait() + ready.Wait() workersDone = make(chan interface{}) - workersAreRestarting.Store(false) + restart.Done() } func getDirectoriesToWatch(workerOpts []workerOpt) []string { @@ -139,10 +144,9 @@ func (worker *worker) startNewThread() { } func (worker *worker) beforeScript(thread *phpThread) { - // if we are restarting due to file watching, wait for all workers to finish first - if workersAreRestarting.Load() { - workerShutdownWG.Done() - workerRestartWG.Wait() + // if we are restarting due to file watching, set the state back to ready + if thread.state.is(stateRestarting) { + thread.state.set(stateReady) } thread.backoff.reset() @@ -245,7 +249,7 @@ func go_frankenphp_worker_handle_request_start(threadIndex C.uintptr_t) C.bool { } // execute opcache_reset if the restart was triggered by the watcher - if watcherIsEnabled && workersAreRestarting.Load() && !executePHPFunction("opcache_reset") { + if watcherIsEnabled && thread.state.is(stateRestarting) && !executePHPFunction("opcache_reset") { logger.Error("failed to call opcache_reset") } From 01ed92bc3becc127639827e5de702fc5051263db Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sun, 17 Nov 2024 22:58:31 +0100 Subject: [PATCH 32/34] Removes unnecessary method. --- thread_state.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/thread_state.go b/thread_state.go index 00540610b3..a669472549 100644 --- a/thread_state.go +++ b/thread_state.go @@ -36,12 +36,6 @@ func (h *threadStateHandler) is(state threadState) bool { return h.currentState == state } -func (h *threadStateHandler) get() threadState { - h.mu.RLock() - defer h.mu.RUnlock() - return h.currentState -} - func (h *threadStateHandler) set(nextState threadState) { h.mu.Lock() defer h.mu.Unlock() From 790cccc1641e555ca4a97d787243cb1a6ab1d8fe Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Sun, 17 Nov 2024 23:15:31 +0100 Subject: [PATCH 33/34] Updates comment. --- thread_state_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/thread_state_test.go b/thread_state_test.go index 10d42635a8..d9ff5fcdb7 100644 --- a/thread_state_test.go +++ b/thread_state_test.go @@ -38,6 +38,6 @@ func TestYieldToAWaitGroupPassedByThreadState(t *testing.T) { threadState.waitFor(stateInactive) threadState.set(stateActive) - // the state should be 'ready' since we are also yielding to the WaitGroup + // 'set' should have yielded to the wait group assert.True(t, hasYielded) } From 0dd26051493dffbdddd6455f5c21f7109f5a12b1 Mon Sep 17 00:00:00 2001 From: Alliballibaba Date: Mon, 18 Nov 2024 09:29:17 +0100 Subject: [PATCH 34/34] Removes unnecessary booleans. --- php_thread.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/php_thread.go b/php_thread.go index bd260f6c70..1692c6d3c2 100644 --- a/php_thread.go +++ b/php_thread.go @@ -5,7 +5,6 @@ import "C" import ( "net/http" "runtime" - "sync/atomic" "unsafe" ) @@ -21,10 +20,6 @@ type phpThread struct { scriptName string // the index in the phpThreads slice threadIndex int - // whether the thread has work assigned to it - isActive atomic.Bool - // whether the thread is ready for work - isReady atomic.Bool // right before the first work iteration onStartup func(*phpThread) // the actual work iteration (done in a loop) @@ -33,8 +28,6 @@ type phpThread struct { afterScriptExecution func(*phpThread, int) // after the thread is done onShutdown func(*phpThread) - // chan to signal the thread to stop the current work iteration - done chan struct{} // exponential backoff for worker failures backoff *exponentialBackoff // known $_SERVER key names