dynamic port selection for temporal (#2865)

This commit is contained in:
Max Novich
2025-06-11 20:25:47 -07:00
committed by GitHub
parent 9432f3c7b5
commit dd5f5eb03d
4 changed files with 535 additions and 98 deletions

View File

@@ -13,9 +13,11 @@ use crate::scheduler::{ScheduledJob, SchedulerError};
use crate::scheduler_trait::SchedulerTrait; use crate::scheduler_trait::SchedulerTrait;
use crate::session::storage::SessionMetadata; use crate::session::storage::SessionMetadata;
const TEMPORAL_SERVICE_URL: &str = "http://localhost:8080"; const TEMPORAL_SERVICE_STARTUP_TIMEOUT: Duration = Duration::from_secs(15);
const TEMPORAL_SERVICE_STARTUP_TIMEOUT: Duration = Duration::from_secs(30); const TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL: Duration = Duration::from_millis(500);
const TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL: Duration = Duration::from_secs(2);
// Default ports to try when discovering the service
const DEFAULT_HTTP_PORTS: &[u16] = &[8080, 8081, 8082, 8083, 8084, 8085];
#[derive(Serialize, Deserialize, Debug)] #[derive(Serialize, Deserialize, Debug)]
struct JobRequest { struct JobRequest {
@@ -50,46 +52,290 @@ struct RunNowResponse {
session_id: String, session_id: String,
} }
#[derive(Serialize, Deserialize, Debug)]
pub struct PortConfig {
http_port: u16,
temporal_port: u16,
ui_port: u16,
}
pub struct TemporalScheduler { pub struct TemporalScheduler {
http_client: Client, http_client: Client,
service_url: String, service_url: String,
port_config: PortConfig,
} }
impl TemporalScheduler { impl TemporalScheduler {
pub async fn new() -> Result<Arc<Self>, SchedulerError> { pub async fn new() -> Result<Arc<Self>, SchedulerError> {
let http_client = Client::new(); let http_client = Client::new();
let service_url = TEMPORAL_SERVICE_URL.to_string();
// Discover the HTTP port
let http_port = Self::discover_http_port(&http_client).await?;
let service_url = format!("http://localhost:{}", http_port);
info!("Found Temporal service HTTP API on port {}", http_port);
// Create scheduler with initial port config
let scheduler = Arc::new(Self { let scheduler = Arc::new(Self {
http_client, http_client: http_client.clone(),
service_url, service_url: service_url.clone(),
port_config: PortConfig {
http_port,
temporal_port: 7233, // temporary defaults
ui_port: 8233,
},
}); });
// Start the Go service (which will handle starting Temporal server) // Start the Go service if not already running
scheduler.start_go_service().await?; scheduler.start_go_service().await?;
// Wait for service to be ready // Wait for service to be ready
scheduler.wait_for_service_ready().await?; scheduler.wait_for_service_ready().await?;
// Fetch the actual port configuration and update
let port_config = scheduler.fetch_port_config().await?;
info!(
"Discovered Temporal service ports - HTTP: {}, Temporal: {}, UI: {}",
port_config.http_port, port_config.temporal_port, port_config.ui_port
);
// Create final scheduler with correct ports
let final_scheduler = Arc::new(Self {
http_client,
service_url,
port_config,
});
info!("TemporalScheduler initialized successfully"); info!("TemporalScheduler initialized successfully");
Ok(scheduler) Ok(final_scheduler)
}
async fn discover_http_port(_http_client: &Client) -> Result<u16, SchedulerError> {
// First, try to find a running service using pgrep and lsof
if let Ok(port) = Self::find_temporal_service_port_from_processes() {
info!(
"Found Temporal service port {} from running processes",
port
);
return Ok(port);
}
// If no running service found, we need to find a free port to start the service on
info!("No running Temporal service found, finding free port to start service");
// Check PORT environment variable first
if let Ok(port_str) = std::env::var("PORT") {
if let Ok(port) = port_str.parse::<u16>() {
if Self::is_port_free(port).await {
info!("Using PORT environment variable: {}", port);
return Ok(port);
} else {
warn!(
"PORT environment variable {} is not free, finding alternative",
port
);
}
}
}
// Try to find a free port from the default list
for &port in DEFAULT_HTTP_PORTS {
if Self::is_port_free(port).await {
info!("Found free port {} for Temporal service", port);
return Ok(port);
}
}
// If all default ports are taken, find any free port in a reasonable range
for port in 8086..8200 {
if Self::is_port_free(port).await {
info!("Found free port {} for Temporal service", port);
return Ok(port);
}
}
Err(SchedulerError::SchedulerInternalError(
"Could not find any free port for Temporal service".to_string(),
))
}
async fn is_port_free(port: u16) -> bool {
use std::net::{SocketAddr, TcpListener};
use std::time::Duration;
let addr: SocketAddr = format!("127.0.0.1:{}", port).parse().unwrap();
// First, try to bind to the port
let listener_result = TcpListener::bind(addr);
match listener_result {
Ok(listener) => {
// Successfully bound, so port was free
drop(listener); // Release the port immediately
// Double-check by trying to connect to see if anything is actually listening
let client = reqwest::Client::builder()
.timeout(Duration::from_millis(500))
.build()
.unwrap();
let test_url = format!("http://127.0.0.1:{}", port);
match client.get(&test_url).send().await {
Ok(_) => {
// Something responded, so port is actually in use
warn!(
"Port {} appeared free but something is listening on it",
port
);
false
}
Err(_) => {
// Nothing responded, port is truly free
true
}
}
}
Err(_) => {
// Could not bind, port is definitely in use
false
}
}
}
fn find_temporal_service_port_from_processes() -> Result<u16, SchedulerError> {
// Use pgrep to find temporal-service processes
let pgrep_output = Command::new("pgrep")
.arg("-f")
.arg("temporal-service")
.output()
.map_err(|e| SchedulerError::SchedulerInternalError(format!("pgrep failed: {}", e)))?;
if !pgrep_output.status.success() {
return Err(SchedulerError::SchedulerInternalError(
"No temporal-service processes found".to_string(),
));
}
let pids_str = String::from_utf8_lossy(&pgrep_output.stdout);
let pids: Vec<&str> = pids_str
.trim()
.split('\n')
.filter(|s| !s.is_empty())
.collect();
for pid in pids {
// Use lsof to find listening ports for this PID
let lsof_output = Command::new("lsof")
.arg("-p")
.arg(pid)
.arg("-i")
.arg("tcp")
.arg("-P") // Show port numbers instead of service names
.arg("-n") // Show IP addresses instead of hostnames
.output();
if let Ok(output) = lsof_output {
let lsof_str = String::from_utf8_lossy(&output.stdout);
// Look for HTTP API port (typically 8080-8999 range)
for line in lsof_str.lines() {
if line.contains("LISTEN") && line.contains("temporal-") {
// Parse lines like: "temporal-service 12345 user 6u IPv4 0x... 0t0 TCP *:8081 (LISTEN)"
let parts: Vec<&str> = line.split_whitespace().collect();
// Find the TCP part which contains the port
for part in &parts {
if part.starts_with("TCP") && part.contains(':') {
// Extract port from TCP *:8081 or TCP 127.0.0.1:8081
if let Some(port_str) = part.split(':').next_back() {
if let Ok(port) = port_str.parse::<u16>() {
// HTTP API ports are typically in 8080-8999 range
if (8080..9000).contains(&port) {
info!("Found HTTP API port {} for PID {}", port, pid);
return Ok(port);
}
}
}
}
}
}
}
}
}
Err(SchedulerError::SchedulerInternalError(
"Could not find HTTP API port from temporal-service processes".to_string(),
))
}
async fn fetch_port_config(&self) -> Result<PortConfig, SchedulerError> {
let url = format!("{}/ports", self.service_url);
match self.http_client.get(&url).send().await {
Ok(response) => {
if response.status().is_success() {
let port_config: PortConfig = response.json().await.map_err(|e| {
SchedulerError::SchedulerInternalError(format!(
"Failed to parse port config JSON: {}",
e
))
})?;
Ok(port_config)
} else {
Err(SchedulerError::SchedulerInternalError(format!(
"Failed to fetch port config: HTTP {}",
response.status()
)))
}
}
Err(e) => Err(SchedulerError::SchedulerInternalError(format!(
"Failed to fetch port config: {}",
e
))),
}
}
/// Get the current port configuration
pub fn get_port_config(&self) -> &PortConfig {
&self.port_config
}
/// Get the Temporal server port
pub fn get_temporal_port(&self) -> u16 {
self.port_config.temporal_port
}
/// Get the HTTP API port
pub fn get_http_port(&self) -> u16 {
self.port_config.http_port
}
/// Get the Temporal UI port
pub fn get_ui_port(&self) -> u16 {
self.port_config.ui_port
} }
async fn start_go_service(&self) -> Result<(), SchedulerError> { async fn start_go_service(&self) -> Result<(), SchedulerError> {
info!("Starting Temporal Go service..."); info!(
"Starting Temporal Go service on port {}...",
self.port_config.http_port
);
// Check if port 8080 is already in use // Check if the service is already running on the discovered port
if self.check_port_in_use(8080).await { if self.health_check().await.unwrap_or(false) {
// Port is in use - check if it's our Go service we can connect to info!(
if self.health_check().await.unwrap_or(false) { "Temporal service is already running on port {}",
info!("Port 8080 is in use by a Go service we can connect to"); self.port_config.http_port
return Ok(()); );
} else { return Ok(());
return Err(SchedulerError::SchedulerInternalError( }
"Port 8080 is already in use by something other than our Go service."
.to_string(), // Double-check that the port is still free (in case something grabbed it between discovery and start)
)); if !Self::is_port_free(self.port_config.http_port).await {
} return Err(SchedulerError::SchedulerInternalError(format!(
"Port {} is no longer available for Temporal service.",
self.port_config.http_port
)));
} }
// Check if the temporal-service binary exists - try multiple possible locations // Check if the temporal-service binary exists - try multiple possible locations
@@ -103,43 +349,70 @@ impl TemporalScheduler {
info!("Found Go service binary at: {}", binary_path); info!("Found Go service binary at: {}", binary_path);
info!("Using working directory: {}", working_dir.display()); info!("Using working directory: {}", working_dir.display());
let command = format!( // Set the PORT environment variable for the service to use and properly daemonize it
"cd '{}' && nohup ./temporal-service > temporal-service.log 2>&1 & echo $!", // Create a new process group to ensure the service survives parent termination
working_dir.display() let mut command = Command::new("./temporal-service");
); command
.current_dir(working_dir)
.env("PORT", self.port_config.http_port.to_string())
.stdout(std::process::Stdio::null())
.stderr(std::process::Stdio::null())
.stdin(std::process::Stdio::null());
let output = Command::new("sh") // On Unix systems, create a new process group
.arg("-c") #[cfg(unix)]
.arg(&command) {
.output() use std::os::unix::process::CommandExt;
.map_err(|e| { command.process_group(0);
SchedulerError::SchedulerInternalError(format!(
"Failed to start Go temporal service: {}",
e
))
})?;
if !output.status.success() {
return Err(SchedulerError::SchedulerInternalError(format!(
"Failed to start Go service: {}",
String::from_utf8_lossy(&output.stderr)
)));
} }
let pid_output = String::from_utf8_lossy(&output.stdout); let child = command.spawn().map_err(|e| {
let pid = pid_output.trim(); SchedulerError::SchedulerInternalError(format!(
info!("Temporal Go service started with PID: {}", pid); "Failed to start Go temporal service: {}",
e
))
})?;
let pid = child.id();
info!(
"Temporal Go service started with PID: {} on port {} (detached)",
pid, self.port_config.http_port
);
// Don't wait for the child process - let it run independently
std::mem::forget(child);
// Give the process a moment to start up
sleep(Duration::from_millis(100)).await;
// Verify the process is still running
#[cfg(unix)]
{
use std::process::Command as StdCommand;
let ps_check = StdCommand::new("ps")
.arg("-p")
.arg(pid.to_string())
.output();
match ps_check {
Ok(output) if output.status.success() => {
info!("Confirmed Temporal service process {} is running", pid);
}
Ok(_) => {
warn!(
"Temporal service process {} may have exited immediately",
pid
);
}
Err(e) => {
warn!("Could not verify Temporal service process status: {}", e);
}
}
}
Ok(()) Ok(())
} }
async fn check_port_in_use(&self, port: u16) -> bool {
use std::net::{SocketAddr, TcpListener};
let addr: SocketAddr = format!("127.0.0.1:{}", port).parse().unwrap();
TcpListener::bind(addr).is_err()
}
fn find_go_service_binary() -> Result<String, SchedulerError> { fn find_go_service_binary() -> Result<String, SchedulerError> {
// Try to find the Go service binary by looking for it relative to the current executable // Try to find the Go service binary by looking for it relative to the current executable
// or in common locations // or in common locations
@@ -191,27 +464,50 @@ impl TemporalScheduler {
info!("Waiting for Temporal service to be ready..."); info!("Waiting for Temporal service to be ready...");
let start_time = std::time::Instant::now(); let start_time = std::time::Instant::now();
let mut attempt_count = 0;
while start_time.elapsed() < TEMPORAL_SERVICE_STARTUP_TIMEOUT { while start_time.elapsed() < TEMPORAL_SERVICE_STARTUP_TIMEOUT {
attempt_count += 1;
match self.health_check().await { match self.health_check().await {
Ok(true) => { Ok(true) => {
info!("Temporal service is ready"); info!(
"Temporal service is ready after {} attempts in {:.2}s",
attempt_count,
start_time.elapsed().as_secs_f64()
);
return Ok(()); return Ok(());
} }
Ok(false) => { Ok(false) => {
// Service responded but not healthy // Service responded but not healthy
if attempt_count % 10 == 0 {
info!(
"Waiting for Temporal service... attempt {} ({:.1}s elapsed)",
attempt_count,
start_time.elapsed().as_secs_f64()
);
}
sleep(TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL).await; sleep(TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL).await;
} }
Err(_) => { Err(e) => {
// Service not responding yet // Service not responding yet
if attempt_count % 10 == 0 {
info!(
"Temporal service not responding yet... attempt {} ({:.1}s elapsed): {}",
attempt_count,
start_time.elapsed().as_secs_f64(),
e
);
}
sleep(TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL).await; sleep(TEMPORAL_SERVICE_HEALTH_CHECK_INTERVAL).await;
} }
} }
} }
Err(SchedulerError::SchedulerInternalError( Err(SchedulerError::SchedulerInternalError(format!(
"Temporal service failed to become ready within timeout".to_string(), "Temporal service failed to become ready within {}s timeout ({} attempts)",
)) TEMPORAL_SERVICE_STARTUP_TIMEOUT.as_secs(),
attempt_count
)))
} }
async fn health_check(&self) -> Result<bool, SchedulerError> { async fn health_check(&self) -> Result<bool, SchedulerError> {
@@ -516,19 +812,19 @@ impl TemporalScheduler {
format!( format!(
"Temporal Services Status:\n\ "Temporal Services Status:\n\
- Go Service ({}:8080): {}\n\ - Go Service (localhost:{}): {}\n\
- Temporal Server (localhost:{}): Running via Go service\n\
- Temporal UI: http://localhost:{}\n\
- Service logs: temporal-service/temporal-service.log\n\ - Service logs: temporal-service/temporal-service.log\n\
- Note: Temporal server is managed by the Go service", - Note: All ports are dynamically allocated",
if go_running { self.port_config.http_port,
"localhost"
} else {
"not running"
},
if go_running { if go_running {
"✅ Running" "✅ Running"
} else { } else {
"❌ Not Running" "❌ Not Running"
} },
self.port_config.temporal_port,
self.port_config.ui_port
) )
} }
@@ -745,16 +1041,11 @@ mod tests {
let rt = Runtime::new().unwrap(); let rt = Runtime::new().unwrap();
rt.block_on(async { rt.block_on(async {
let scheduler = TemporalScheduler {
http_client: reqwest::Client::new(),
service_url: "http://localhost:8080".to_string(),
};
// Test with a port that should be available (high port number) // Test with a port that should be available (high port number)
let high_port_in_use = scheduler.check_port_in_use(65432).await; let high_port_in_use = !TemporalScheduler::is_port_free(65432).await;
// Test with a port that might be in use (port 80) // Test with a port that might be in use (port 80)
let low_port_in_use = scheduler.check_port_in_use(80).await; let low_port_in_use = !TemporalScheduler::is_port_free(80).await;
println!("✅ Port checking functionality works"); println!("✅ Port checking functionality works");
println!(" High port (65432) in use: {}", high_port_in_use); println!(" High port (65432) in use: {}", high_port_in_use);
@@ -779,4 +1070,38 @@ mod tests {
} }
} }
} }
#[test]
fn test_daemon_process_group_creation() {
// Test that the daemon process creation logic compiles and works correctly
use std::process::Command;
// Create a test command similar to what we do in start_go_service
let mut command = Command::new("echo");
command
.arg("test")
.stdout(std::process::Stdio::null())
.stderr(std::process::Stdio::null())
.stdin(std::process::Stdio::null());
// On Unix systems, create a new process group
#[cfg(unix)]
{
use std::os::unix::process::CommandExt;
command.process_group(0);
}
// Test that the command can be spawned (but don't actually run it)
match command.spawn() {
Ok(mut child) => {
println!("✅ Daemon process group creation works");
// Clean up the test process
let _ = child.wait();
}
Err(e) => {
println!("⚠️ Error spawning test process: {}", e);
// This might happen in some test environments, but the logic is correct
}
}
}
} }

View File

@@ -5,11 +5,13 @@ import (
"encoding/json" "encoding/json"
"fmt" "fmt"
"log" "log"
"net"
"net/http" "net/http"
"os" "os"
"os/exec" "os/exec"
"os/signal" "os/signal"
"path/filepath" "path/filepath"
"strconv"
"strings" "strings"
"syscall" "syscall"
"time" "time"
@@ -27,6 +29,59 @@ const (
Namespace = "default" Namespace = "default"
) )
// PortConfig holds the port configuration for Temporal services
type PortConfig struct {
TemporalPort int // Main Temporal server port
UIPort int // Temporal UI port
HTTPPort int // HTTP API port
}
// findAvailablePort finds an available port starting from the given port
func findAvailablePort(startPort int) (int, error) {
for port := startPort; port < startPort+100; port++ {
ln, err := net.Listen("tcp", fmt.Sprintf(":%d", port))
if err == nil {
ln.Close()
return port, nil
}
}
return 0, fmt.Errorf("no available port found starting from %d", startPort)
}
// findAvailablePorts finds available ports for all Temporal services
func findAvailablePorts() (*PortConfig, error) {
// Try to find available ports starting from preferred defaults
temporalPort, err := findAvailablePort(7233)
if err != nil {
return nil, fmt.Errorf("failed to find available port for Temporal server: %w", err)
}
uiPort, err := findAvailablePort(8233)
if err != nil {
return nil, fmt.Errorf("failed to find available port for Temporal UI: %w", err)
}
// For HTTP port, check environment variable first
httpPort := 8080
if portEnv := os.Getenv("PORT"); portEnv != "" {
if parsed, err := strconv.Atoi(portEnv); err == nil {
httpPort = parsed
}
}
// Verify HTTP port is available, find alternative if not
finalHTTPPort, err := findAvailablePort(httpPort)
if err != nil {
return nil, fmt.Errorf("failed to find available port for HTTP server: %w", err)
}
return &PortConfig{
TemporalPort: temporalPort,
UIPort: uiPort,
HTTPPort: finalHTTPPort,
}, nil
}
// Global service instance for activities to access // Global service instance for activities to access
var globalService *TemporalService var globalService *TemporalService
@@ -61,16 +116,16 @@ type RunNowResponse struct {
} }
// ensureTemporalServerRunning checks if Temporal server is running and starts it if needed // ensureTemporalServerRunning checks if Temporal server is running and starts it if needed
func ensureTemporalServerRunning() error { func ensureTemporalServerRunning(ports *PortConfig) error {
log.Println("Checking if Temporal server is running...") log.Println("Checking if Temporal server is running...")
// Check if Temporal server is already running by trying to connect // Check if Temporal server is already running by trying to connect
if isTemporalServerRunning() { if isTemporalServerRunning(ports.TemporalPort) {
log.Println("Temporal server is already running") log.Printf("Temporal server is already running on port %d", ports.TemporalPort)
return nil return nil
} }
log.Println("Temporal server not running, attempting to start it...") log.Printf("Temporal server not running, attempting to start it on port %d...", ports.TemporalPort)
// Find the temporal CLI binary // Find the temporal CLI binary
temporalCmd, err := findTemporalCLI() temporalCmd, err := findTemporalCLI()
@@ -83,8 +138,8 @@ func ensureTemporalServerRunning() error {
// Start Temporal server in background // Start Temporal server in background
cmd := exec.Command(temporalCmd, "server", "start-dev", cmd := exec.Command(temporalCmd, "server", "start-dev",
"--db-filename", "temporal.db", "--db-filename", "temporal.db",
"--port", "7233", "--port", strconv.Itoa(ports.TemporalPort),
"--ui-port", "8233", "--ui-port", strconv.Itoa(ports.UIPort),
"--log-level", "warn") "--log-level", "warn")
// Start the process in background // Start the process in background
@@ -92,7 +147,8 @@ func ensureTemporalServerRunning() error {
return fmt.Errorf("failed to start Temporal server: %w", err) return fmt.Errorf("failed to start Temporal server: %w", err)
} }
log.Printf("Temporal server started with PID: %d", cmd.Process.Pid) log.Printf("Temporal server started with PID: %d (port: %d, UI port: %d)",
cmd.Process.Pid, ports.TemporalPort, ports.UIPort)
// Wait for server to be ready (with timeout) // Wait for server to be ready (with timeout)
timeout := time.After(30 * time.Second) timeout := time.After(30 * time.Second)
@@ -104,8 +160,8 @@ func ensureTemporalServerRunning() error {
case <-timeout: case <-timeout:
return fmt.Errorf("timeout waiting for Temporal server to start") return fmt.Errorf("timeout waiting for Temporal server to start")
case <-ticker.C: case <-ticker.C:
if isTemporalServerRunning() { if isTemporalServerRunning(ports.TemporalPort) {
log.Println("Temporal server is now ready") log.Printf("Temporal server is now ready on port %d", ports.TemporalPort)
return nil return nil
} }
} }
@@ -113,10 +169,10 @@ func ensureTemporalServerRunning() error {
} }
// isTemporalServerRunning checks if Temporal server is accessible // isTemporalServerRunning checks if Temporal server is accessible
func isTemporalServerRunning() bool { func isTemporalServerRunning(port int) bool {
// Try to create a client connection to check if server is running // Try to create a client connection to check if server is running
c, err := client.Dial(client.Options{ c, err := client.Dial(client.Options{
HostPort: "127.0.0.1:7233", HostPort: fmt.Sprintf("127.0.0.1:%d", port),
Namespace: Namespace, Namespace: Namespace,
}) })
if err != nil { if err != nil {
@@ -143,6 +199,19 @@ func findTemporalCLI() (string, error) {
} }
} }
// Try using 'which' command to find temporal
cmd := exec.Command("which", "temporal")
if output, err := cmd.Output(); err == nil {
path := strings.TrimSpace(string(output))
if path != "" {
// Verify it's the correct temporal CLI by checking version
cmd := exec.Command(path, "--version")
if err := cmd.Run(); err == nil {
return path, nil
}
}
}
// If not found in PATH, try different possible locations for the temporal CLI // If not found in PATH, try different possible locations for the temporal CLI
possiblePaths := []string{ possiblePaths := []string{
"./temporal", // Current directory "./temporal", // Current directory
@@ -180,18 +249,28 @@ type TemporalService struct {
worker worker.Worker worker worker.Worker
scheduleJobs map[string]*JobStatus // In-memory job tracking scheduleJobs map[string]*JobStatus // In-memory job tracking
runningJobs map[string]bool // Track which jobs are currently running runningJobs map[string]bool // Track which jobs are currently running
ports *PortConfig // Port configuration
} }
// NewTemporalService creates a new Temporal service and ensures Temporal server is running // NewTemporalService creates a new Temporal service and ensures Temporal server is running
func NewTemporalService() (*TemporalService, error) { func NewTemporalService() (*TemporalService, error) {
// First, ensure Temporal server is running // First, find available ports
if err := ensureTemporalServerRunning(); err != nil { ports, err := findAvailablePorts()
if err != nil {
return nil, fmt.Errorf("failed to find available ports: %w", err)
}
log.Printf("Using ports - Temporal: %d, UI: %d, HTTP: %d",
ports.TemporalPort, ports.UIPort, ports.HTTPPort)
// Ensure Temporal server is running
if err := ensureTemporalServerRunning(ports); err != nil {
return nil, fmt.Errorf("failed to ensure Temporal server is running: %w", err) return nil, fmt.Errorf("failed to ensure Temporal server is running: %w", err)
} }
// Create client (Temporal server should now be running) // Create client (Temporal server should now be running)
c, err := client.Dial(client.Options{ c, err := client.Dial(client.Options{
HostPort: "127.0.0.1:7233", HostPort: fmt.Sprintf("127.0.0.1:%d", ports.TemporalPort),
Namespace: Namespace, Namespace: Namespace,
}) })
if err != nil { if err != nil {
@@ -208,13 +287,14 @@ func NewTemporalService() (*TemporalService, error) {
return nil, fmt.Errorf("failed to start worker: %w", err) return nil, fmt.Errorf("failed to start worker: %w", err)
} }
log.Println("Connected to Temporal server successfully") log.Printf("Connected to Temporal server successfully on port %d", ports.TemporalPort)
service := &TemporalService{ service := &TemporalService{
client: c, client: c,
worker: w, worker: w,
scheduleJobs: make(map[string]*JobStatus), scheduleJobs: make(map[string]*JobStatus),
runningJobs: make(map[string]bool), runningJobs: make(map[string]bool),
ports: ports,
} }
// Set global service for activities // Set global service for activities
@@ -235,6 +315,21 @@ func (ts *TemporalService) Stop() {
log.Println("Temporal service stopped") log.Println("Temporal service stopped")
} }
// GetHTTPPort returns the HTTP port for this service
func (ts *TemporalService) GetHTTPPort() int {
return ts.ports.HTTPPort
}
// GetTemporalPort returns the Temporal server port for this service
func (ts *TemporalService) GetTemporalPort() int {
return ts.ports.TemporalPort
}
// GetUIPort returns the Temporal UI port for this service
func (ts *TemporalService) GetUIPort() int {
return ts.ports.UIPort
}
// Workflow definition for executing Goose recipes // Workflow definition for executing Goose recipes
func GooseJobWorkflow(ctx workflow.Context, jobID, recipePath string) (string, error) { func GooseJobWorkflow(ctx workflow.Context, jobID, recipePath string) (string, error) {
logger := workflow.GetLogger(ctx) logger := workflow.GetLogger(ctx)
@@ -610,29 +705,45 @@ func (ts *TemporalService) handleHealth(w http.ResponseWriter, r *http.Request)
json.NewEncoder(w).Encode(map[string]string{"status": "healthy"}) json.NewEncoder(w).Encode(map[string]string{"status": "healthy"})
} }
func main() { // handlePorts returns the port configuration for this service
port := os.Getenv("PORT") func (ts *TemporalService) handlePorts(w http.ResponseWriter, r *http.Request) {
if port == "" { w.Header().Set("Content-Type", "application/json")
port = "8080" w.WriteHeader(http.StatusOK)
portInfo := map[string]int{
"http_port": ts.ports.HTTPPort,
"temporal_port": ts.ports.TemporalPort,
"ui_port": ts.ports.UIPort,
} }
log.Println("Starting Temporal service...") json.NewEncoder(w).Encode(portInfo)
log.Println("Note: This service requires a running Temporal server at 127.0.0.1:7233") }
log.Println("Start Temporal server with: temporal server start-dev")
// Create Temporal service func main() {
log.Println("Starting Temporal service...")
// Create Temporal service (this will find available ports automatically)
service, err := NewTemporalService() service, err := NewTemporalService()
if err != nil { if err != nil {
log.Fatalf("Failed to create Temporal service: %v", err) log.Fatalf("Failed to create Temporal service: %v", err)
} }
// Use the dynamically assigned HTTP port
httpPort := service.GetHTTPPort()
temporalPort := service.GetTemporalPort()
uiPort := service.GetUIPort()
log.Printf("Temporal server running on port %d", temporalPort)
log.Printf("Temporal UI available at http://localhost:%d", uiPort)
// Set up HTTP server // Set up HTTP server
mux := http.NewServeMux() mux := http.NewServeMux()
mux.HandleFunc("/jobs", service.handleJobs) mux.HandleFunc("/jobs", service.handleJobs)
mux.HandleFunc("/health", service.handleHealth) mux.HandleFunc("/health", service.handleHealth)
mux.HandleFunc("/ports", service.handlePorts)
server := &http.Server{ server := &http.Server{
Addr: ":" + port, Addr: fmt.Sprintf(":%d", httpPort),
Handler: mux, Handler: mux,
} }
@@ -655,9 +766,10 @@ func main() {
os.Exit(0) os.Exit(0)
}() }()
log.Printf("Temporal service starting on port %s", port) log.Printf("Temporal service starting on port %d", httpPort)
log.Printf("Health endpoint: http://localhost:%s/health", port) log.Printf("Health endpoint: http://localhost:%d/health", httpPort)
log.Printf("Jobs endpoint: http://localhost:%s/jobs", port) log.Printf("Jobs endpoint: http://localhost:%d/jobs", httpPort)
log.Printf("Ports endpoint: http://localhost:%d/ports", httpPort)
if err := server.ListenAndServe(); err != nil && err != http.ErrServerClosed { if err := server.ListenAndServe(); err != nil && err != http.ErrServerClosed {
log.Fatalf("HTTP server failed: %v", err) log.Fatalf("HTTP server failed: %v", err)

Binary file not shown.

View File

@@ -26,7 +26,7 @@ export const findAvailablePort = (): Promise<number> => {
// Check if goosed server is ready by polling the status endpoint // Check if goosed server is ready by polling the status endpoint
const checkServerStatus = async ( const checkServerStatus = async (
port: number, port: number,
maxAttempts: number = 60, maxAttempts: number = 80,
interval: number = 100 interval: number = 100
): Promise<boolean> => { ): Promise<boolean> => {
const statusUrl = `http://127.0.0.1:${port}/status`; const statusUrl = `http://127.0.0.1:${port}/status`;