summaryrefslogtreecommitdiff
path: root/Omni
diff options
context:
space:
mode:
authorBen Sima <ben@bensima.com>2025-11-26 20:53:24 -0500
committerBen Sima <ben@bensima.com>2025-11-26 20:53:24 -0500
commit34dbe1dd88bae19b27691e4346421133dea57556 (patch)
tree4bc7793d4781bcbba9cf1936dfa662804b0923dd /Omni
parent0fa8ed4689022cb213de9cfb88a10841c7a03935 (diff)
Bild: pipelined analyze→build architecture with multi-line UI
Implement STM-based pipeline that allows per-target progression through build phases (analyze → wait for deps → build) with concurrent workers. Key changes: - TargetState enum tracks each target through pipeline stages - Coordinator manages state, analyze queue, build queue coordination - pipelineAnalysisWorker/pipelineBuildWorker pull from queues concurrently - promoteWaiters unblocks targets when their dependencies complete UI improvements: - Multi-line mode reserves N lines for N targets, updates in-place - Remove narrow terminal (<80 col) restriction for multi-line mode - Add Skipped state with yellow [_] for non-buildable files - Remove extra blank line at start of output State symbols: [.] Pending, [+] Analyzing, [~] Building, [✓] Success (green), [x] Failed (red), [_] Skipped (yellow)
Diffstat (limited to 'Omni')
-rw-r--r--Omni/Bild.hs759
-rw-r--r--Omni/Log/Concurrent.hs10
-rw-r--r--Omni/Log/Terminal.hs3
3 files changed, 486 insertions, 286 deletions
diff --git a/Omni/Bild.hs b/Omni/Bild.hs
index 588e7f2..c1df8e7 100644
--- a/Omni/Bild.hs
+++ b/Omni/Bild.hs
@@ -120,6 +120,7 @@ import Alpha hiding (sym, (<.>))
import qualified Conduit
import qualified Control.Concurrent.Async as Async
import qualified Control.Concurrent.QSemN as QSemN
+import Control.Concurrent.STM (TQueue, TVar, modifyTVar', newTQueue, newTVar, readTVar, readTVarIO, tryReadTQueue, writeTQueue)
import qualified Control.Exception as Exception
import qualified Data.Aeson as Aeson
import qualified Data.ByteString as ByteString
@@ -132,6 +133,7 @@ import qualified Data.Conduit.Combinators as Conduit
import qualified Data.Conduit.Process as Conduit
import Data.Graph (SCC (..), stronglyConnComp)
import Data.IORef (IORef, modifyIORef', newIORef, readIORef, writeIORef)
+import Data.List (partition)
import qualified Data.List as List
import qualified Data.Map as Map
import qualified Data.Set as Set
@@ -218,37 +220,46 @@ move args = do
IO.hSetBuffering stdout IO.NoBuffering
root <- getCoderoot
loadGhcPkgCache
- namespaces <-
+ allNamespaces <-
Cli.getAllArgs args (Cli.argument "target")
|> filterM Dir.doesFileExist
+> filterGitIgnored
/> filter (\x -> isGitHook x |> don't)
+> traverse Dir.makeAbsolute
+> traverse (namespaceFromPathOrDie root)
- /> filter isBuildableNs
+ let (namespaces, skippedNamespaces) = partition isBuildableNs allNamespaces
let isPlanMode = args `Cli.has` Cli.longOption "plan"
- -- Wrap entire analyze+build sequence in a single LineManager
- -- to show all targets upfront and track status through analyze -> build
- let runWithManager action = case (isPlanMode, isLoud) of
- (True, _) -> action -- Plan mode doesn't need a manager
- (_, True) -> action -- Loud mode doesn't need a manager
- _ ->
- LogC.withLineManager namespaces <| \mgr -> do
- LogC.initializeLines mgr -- Show all targets as pending upfront
- action
- runWithManager <| do
- analysis <- analyzeAll isPlanMode namespaces
- printOrBuild root analysis
- |> Timeout.timeout (toMillis minutes)
- +> \case
- Nothing ->
- Log.br
- >> Log.fail ["bild", "timeout after " <> tshow minutes <> " minutes"]
- >> Log.br
- >> exitWith (ExitFailure 124)
- Just s -> do
- when (all isSuccess s) saveGhcPkgCache
- exitSummary s
+ if isPlanMode
+ then do
+ analysis <- analyzeAll True namespaces
+ if Map.null analysis
+ then Log.wipe >> Log.fail ["bild", "nothing to build"] >> Log.br >> exitWith (ExitFailure 1)
+ else putJSON analysis
+ else do
+ when (null allNamespaces) <| do
+ Log.wipe >> Log.fail ["bild", "nothing to build"] >> Log.br >> exitWith (ExitFailure 1)
+ nproc <- GHC.getNumProcessors
+ createHier root
+ let runWithManager action =
+ if isLoud
+ then action
+ else
+ LogC.withLineManager allNamespaces <| \mgr -> do
+ LogC.initializeLines mgr
+ forM_ skippedNamespaces <| \ns -> LogC.updateLineState ns LogC.Skipped
+ action
+ runWithManager <| do
+ pipelineBuild isTest isLoud 8 jobs (cpus nproc) namespaces analyzeOne
+ |> Timeout.timeout (toMillis minutes)
+ +> \case
+ Nothing ->
+ Log.br
+ >> Log.fail ["bild", "timeout after " <> tshow minutes <> " minutes"]
+ >> Log.br
+ >> exitWith (ExitFailure 124)
+ Just s -> do
+ when (all isSuccess s) saveGhcPkgCache
+ exitSummary s
where
minutes =
Cli.getArgWithDefault args "10" (Cli.longOption "time")
@@ -256,19 +267,6 @@ move args = do
|> \case
Nothing -> panic "could not read --time argument"
Just n -> (n == 0) ?: (-1, n)
- printOrBuild :: FilePath -> Analysis -> IO [ExitCode]
- printOrBuild root targets
- | Map.null targets =
- Log.wipe
- >> Log.fail ["bild", "nothing to build"]
- >> Log.br
- >> exitWith (ExitFailure 1)
- | args `Cli.has` Cli.longOption "plan" =
- putJSON targets >> pure [Exit.ExitSuccess]
- | otherwise = do
- nproc <- GHC.getNumProcessors
- createHier root
- build isTest isLoud jobs (cpus nproc) targets
cpus :: Int -> Int
cpus nproc =
Cli.longOption "cpus"
@@ -564,277 +562,269 @@ type Analysis = Map Namespace Target
analyzeAll :: Bool -> [Namespace] -> IO Analysis
analyzeAll _isPlanMode nss = do
- -- Analysis runs concurrently, updating each namespace's status as it progresses
- -- LineManager is set up by caller (move), so we just update states here
targets <- mapConcurrentlyBounded 8 analyzeOne nss
pure <| Map.fromList <| catMaybes <| zipWith (\ns mt -> (ns,) </ mt) nss targets
- where
- analyzeOne :: Namespace -> IO (Maybe Target)
- analyzeOne namespace@(Namespace parts ext) = do
- LogC.updateLineState namespace LogC.Analyzing
- let path = Namespace.toPath namespace
- root <- getCoderoot
- let abspath = root </> path
- let quapath = path
- user <- Env.getEnv "USER" /> Text.pack
- host <- HostName.getHostName /> Text.pack
- contentLines <-
- withFile abspath ReadMode <| \h ->
- IO.hSetEncoding h IO.utf8_bom
- >> Text.IO.hGetContents h
- /> Text.lines
- -- if the file is exe but doesn't have 'out' metadata, just use the
- -- dot-separated namespace instead
- isExe <- Dir.getPermissions quapath /> Dir.executable
- let defaultOut = isExe ?: (Just <| Namespace.dotSeparated parts, Nothing)
- result <- case ext of
- -- basically we don't support building these
- Namespace.Css -> pure Nothing
- Namespace.Json -> pure Nothing
- Namespace.Keys -> pure Nothing
- Namespace.Md -> pure Nothing
- Namespace.None -> pure Nothing
- Namespace.Html -> pure Nothing
- Namespace.Toml -> pure Nothing
- Namespace.Py ->
- contentLines
- |> Meta.detectAll "#"
- |> \Meta.Parsed {..} ->
- detectPythonImports mempty contentLines +> \srcs ->
- Target
- { builder = "python",
- wrapper = Nothing,
- compiler = CPython,
- compilerFlags =
- -- This doesn't really make sense for python, but I'll leave
- -- it here for eventual --dev builds
- [ "-c",
- "\"import py_compile;import os;"
- <> "py_compile.compile(file='"
- <> str quapath
- <> "', cfile=os.getenv('CODEROOT')+'/_/int/"
- <> str quapath
- <> "', doraise=True)\""
- ],
- sysdeps = psys,
- langdeps = pdep,
- outPath = outToPath pout,
- out = pout <|> defaultOut,
- packageSet = "python.packages",
- mainModule = Namespace.toModule namespace,
- rundeps = prun,
- hsGraph = Nothing,
- ..
- }
- |> Just
- |> pure
- Namespace.Sh -> pure Nothing
- Namespace.C ->
- Meta.detectAll "//" contentLines |> \Meta.Parsed {..} -> do
+
+analyzeOne :: Namespace -> IO (Maybe Target)
+analyzeOne namespace@(Namespace parts ext) = do
+ let path = Namespace.toPath namespace
+ root <- getCoderoot
+ let abspath = root </> path
+ let quapath = path
+ user <- Env.getEnv "USER" /> Text.pack
+ host <- HostName.getHostName /> Text.pack
+ contentLines <-
+ withFile abspath ReadMode <| \h ->
+ IO.hSetEncoding h IO.utf8_bom
+ >> Text.IO.hGetContents h
+ /> Text.lines
+ isExe <- Dir.getPermissions quapath /> Dir.executable
+ let defaultOut = isExe ?: (Just <| Namespace.dotSeparated parts, Nothing)
+ case ext of
+ Namespace.Css -> pure Nothing
+ Namespace.Json -> pure Nothing
+ Namespace.Keys -> pure Nothing
+ Namespace.Md -> pure Nothing
+ Namespace.None -> pure Nothing
+ Namespace.Html -> pure Nothing
+ Namespace.Toml -> pure Nothing
+ Namespace.Py ->
+ contentLines
+ |> Meta.detectAll "#"
+ |> \Meta.Parsed {..} ->
+ detectPythonImports mempty contentLines +> \srcs ->
Target
- { langdeps = pdep,
- sysdeps = psys,
+ { builder = "python",
wrapper = Nothing,
- compiler = Gcc,
- builder = "c",
+ compiler = CPython,
+ compilerFlags =
+ -- This doesn't really make sense for python, but I'll leave
+ -- it here for eventual --dev builds
+ [ "-c",
+ "\"import py_compile;import os;"
+ <> "py_compile.compile(file='"
+ <> str quapath
+ <> "', cfile=os.getenv('CODEROOT')+'/_/int/"
+ <> str quapath
+ <> "', doraise=True)\""
+ ],
+ sysdeps = psys,
+ langdeps = pdep,
+ outPath = outToPath pout,
out = pout <|> defaultOut,
- packageSet = "c.packages",
+ packageSet = "python.packages",
mainModule = Namespace.toModule namespace,
- compilerFlags = case pout of
- Just o ->
- ["-o", o, path] <> Set.toList parg |> map Text.pack
- Nothing -> panic "can only bild C exes, not libs",
- outPath = outToPath pout,
- -- implement detectCImports, then I can fill this out
- srcs = Set.empty,
rundeps = prun,
hsGraph = Nothing,
..
}
|> Just
|> pure
- Namespace.Hs ->
- contentLines
- |> Meta.detectAll "--"
- |> \Meta.Parsed {..} ->
- detectHaskellImports mempty contentLines +> \(autoDeps, srcs) -> do
- let langdeps = autoDeps <> pdep
- graph <- buildHsModuleGraph namespace quapath srcs
- pure
- <| Just
- Target
- { builder = "haskell",
- wrapper = Nothing,
- compiler = Ghc,
- packageSet = "haskell.packages",
- mainModule = Namespace.toModule namespace,
- compilerFlags =
- [ "-Wall",
- "-Werror",
- "-haddock",
- "-Winvalid-haddock",
- "-threaded",
- "-i$CODEROOT",
- "-odir",
- ".",
- "-hidir",
- ".",
- "--make",
- "$CODEROOT" </> quapath
- ]
- ++ case pout of
- Just o ->
- [ "-main-is",
- Namespace.toHaskellModule namespace,
- "-o",
- o
- ]
- Nothing -> []
- |> map Text.pack,
- sysdeps = Meta.detect (Meta.sys "--") contentLines,
- outPath = outToPath pout,
- rundeps = prun,
- out = pout <|> defaultOut,
- hsGraph = graph,
- ..
- }
- Namespace.Lisp ->
- Meta.detectOut (Meta.out ";;") contentLines |> \out -> do
- langdeps <- detectLispImports contentLines
- Just
- </ pure
+ Namespace.Sh -> pure Nothing
+ Namespace.C ->
+ Meta.detectAll "//" contentLines |> \Meta.Parsed {..} -> do
+ Target
+ { langdeps = pdep,
+ sysdeps = psys,
+ wrapper = Nothing,
+ compiler = Gcc,
+ builder = "c",
+ out = pout <|> defaultOut,
+ packageSet = "c.packages",
+ mainModule = Namespace.toModule namespace,
+ compilerFlags = case pout of
+ Just o ->
+ ["-o", o, path] <> Set.toList parg |> map Text.pack
+ Nothing -> panic "can only bild C exes, not libs",
+ outPath = outToPath pout,
+ -- implement detectCImports, then I can fill this out
+ srcs = Set.empty,
+ rundeps = prun,
+ hsGraph = Nothing,
+ ..
+ }
+ |> Just
+ |> pure
+ Namespace.Hs ->
+ contentLines
+ |> Meta.detectAll "--"
+ |> \Meta.Parsed {..} ->
+ detectHaskellImports mempty contentLines +> \(autoDeps, srcs) -> do
+ let langdeps = autoDeps <> pdep
+ graph <- buildHsModuleGraph namespace quapath srcs
+ pure
+ <| Just
Target
- { sysdeps = Set.empty,
+ { builder = "haskell",
wrapper = Nothing,
- compiler = Sbcl,
- packageSet = "lisp.sbclWith",
+ compiler = Ghc,
+ packageSet = "haskell.packages",
mainModule = Namespace.toModule namespace,
compilerFlags =
- map
- Text.pack
- [ "--eval",
- "(require :asdf)",
- "--load",
- quapath,
- "--eval",
- "(sb-ext:save-lisp-and-die #p\"" <> (root </> outToPath out) <> "\" :toplevel #'main :executable t)"
- ],
- builder = "base",
- outPath = outToPath out,
- -- add local src imports to detectLispImports, then i can fill this out
- srcs = Set.empty,
- rundeps = Set.empty,
- hsGraph = Nothing,
+ [ "-Wall",
+ "-Werror",
+ "-haddock",
+ "-Winvalid-haddock",
+ "-threaded",
+ "-i$CODEROOT",
+ "-odir",
+ ".",
+ "-hidir",
+ ".",
+ "--make",
+ "$CODEROOT" </> quapath
+ ]
+ ++ case pout of
+ Just o ->
+ [ "-main-is",
+ Namespace.toHaskellModule namespace,
+ "-o",
+ o
+ ]
+ Nothing -> []
+ |> map Text.pack,
+ sysdeps = Meta.detect (Meta.sys "--") contentLines,
+ outPath = outToPath pout,
+ rundeps = prun,
+ out = pout <|> defaultOut,
+ hsGraph = graph,
..
}
- Namespace.Nix ->
- (host == "lithium") ?: (Local user "lithium", Remote user "dev.bensima.com") |> \builder ->
+ Namespace.Lisp ->
+ Meta.detectOut (Meta.out ";;") contentLines |> \out -> do
+ langdeps <- detectLispImports contentLines
+ Just
+ </ pure
Target
- { langdeps = Set.empty,
+ { sysdeps = Set.empty,
wrapper = Nothing,
- sysdeps = Set.empty,
- compiler = NixBuild,
- compilerFlags =
- [ quapath,
- "--out-link",
- root </> nixdir </> Namespace.toPath namespace,
- "--builders",
- toNixFlag builder,
- "--arg",
- "bild",
- str <| "import " <> root </> "Omni/Bild.nix {}"
- ]
- |> map Text.pack,
- out = Nothing,
- outPath = outToPath Nothing,
- srcs = Set.empty,
- packageSet = "",
- mainModule = Namespace.toModule namespace,
- builder = "base",
- rundeps = Set.empty,
- hsGraph = Nothing,
- ..
- }
- |> Just
- |> pure
- Namespace.Scm ->
- Meta.detectAll ";;" contentLines |> \Meta.Parsed {..} ->
- Target
- { langdeps = pdep,
- sysdeps = psys,
- compiler = Guile,
- packageSet = "scheme.guilePackages",
+ compiler = Sbcl,
+ packageSet = "lisp.sbclWith",
mainModule = Namespace.toModule namespace,
compilerFlags =
- [ "compile",
- "--r7rs",
- "--load-path=" ++ root,
- "--output=" ++ root </> intdir </> replaceExtension quapath ".scm.go",
- quapath
- ]
- |> map Text.pack,
+ map
+ Text.pack
+ [ "--eval",
+ "(require :asdf)",
+ "--load",
+ quapath,
+ "--eval",
+ "(sb-ext:save-lisp-and-die #p\"" <> (root </> outToPath out) <> "\" :toplevel #'main :executable t)"
+ ],
builder = "base",
- outPath = outToPath pout,
- out = pout <|> defaultOut,
- srcs = Set.empty, -- implement detectSchemeImports
- -- TODO: wrapper should just be removed, instead rely on
- -- upstream nixpkgs builders to make wrappers
- wrapper =
- isNothing pout
- ?: ( Nothing,
- [ "#!/usr/bin/env bash",
- "guile -C \""
- <> root
- </> intdir
- <> "\" -e main "
- <> "-s "
- <> Namespace.toPath namespace
- <> " \"$@\""
- ]
- |> joinWith "\n"
- |> Text.pack
- |> Just
- ),
- rundeps = prun,
- hsGraph = Nothing,
- ..
- }
- |> Just
- |> pure
- Namespace.Rs ->
- Meta.detectAll "//" contentLines |> \Meta.Parsed {..} ->
- Target
- { langdeps = pdep,
- -- this packageSet doesn't actually exist because everyone in
- -- nix just generates nix expressions for rust dependencies with
- -- Cargo.lock, so I have to make it in order to use rust deps
- packageSet = "rust.packages",
- mainModule = Namespace.toModule namespace,
- wrapper = Nothing,
- sysdeps = psys <> Set.singleton "rustc",
- out = pout <|> defaultOut,
- compiler = Rustc,
- compilerFlags = case pout of
- Just o ->
- map
- Text.pack
- [ "$CODEROOT" </> path,
- "-o",
- o
- ]
- Nothing -> panic "can't build rust libs",
- builder = "base",
- outPath = outToPath pout,
- -- implement detectRustImports
+ outPath = outToPath out,
+ -- add local src imports to detectLispImports, then i can fill this out
srcs = Set.empty,
- rundeps = prun,
+ rundeps = Set.empty,
hsGraph = Nothing,
..
}
- |> Just
- |> pure
- LogC.updateLineState namespace LogC.Pending
- pure result
+ Namespace.Nix ->
+ (host == "lithium") ?: (Local user "lithium", Remote user "dev.bensima.com") |> \builder ->
+ Target
+ { langdeps = Set.empty,
+ wrapper = Nothing,
+ sysdeps = Set.empty,
+ compiler = NixBuild,
+ compilerFlags =
+ [ quapath,
+ "--out-link",
+ root </> nixdir </> Namespace.toPath namespace,
+ "--builders",
+ toNixFlag builder,
+ "--arg",
+ "bild",
+ str <| "import " <> root </> "Omni/Bild.nix {}"
+ ]
+ |> map Text.pack,
+ out = Nothing,
+ outPath = outToPath Nothing,
+ srcs = Set.empty,
+ packageSet = "",
+ mainModule = Namespace.toModule namespace,
+ builder = "base",
+ rundeps = Set.empty,
+ hsGraph = Nothing,
+ ..
+ }
+ |> Just
+ |> pure
+ Namespace.Scm ->
+ Meta.detectAll ";;" contentLines |> \Meta.Parsed {..} ->
+ Target
+ { langdeps = pdep,
+ sysdeps = psys,
+ compiler = Guile,
+ packageSet = "scheme.guilePackages",
+ mainModule = Namespace.toModule namespace,
+ compilerFlags =
+ [ "compile",
+ "--r7rs",
+ "--load-path=" ++ root,
+ "--output=" ++ root </> intdir </> replaceExtension quapath ".scm.go",
+ quapath
+ ]
+ |> map Text.pack,
+ builder = "base",
+ outPath = outToPath pout,
+ out = pout <|> defaultOut,
+ srcs = Set.empty, -- implement detectSchemeImports
+ -- TODO: wrapper should just be removed, instead rely on
+ -- upstream nixpkgs builders to make wrappers
+ wrapper =
+ isNothing pout
+ ?: ( Nothing,
+ [ "#!/usr/bin/env bash",
+ "guile -C \""
+ <> root
+ </> intdir
+ <> "\" -e main "
+ <> "-s "
+ <> Namespace.toPath namespace
+ <> " \"$@\""
+ ]
+ |> joinWith "\n"
+ |> Text.pack
+ |> Just
+ ),
+ rundeps = prun,
+ hsGraph = Nothing,
+ ..
+ }
+ |> Just
+ |> pure
+ Namespace.Rs ->
+ Meta.detectAll "//" contentLines |> \Meta.Parsed {..} ->
+ Target
+ { langdeps = pdep,
+ -- this packageSet doesn't actually exist because everyone in
+ -- nix just generates nix expressions for rust dependencies with
+ -- Cargo.lock, so I have to make it in order to use rust deps
+ packageSet = "rust.packages",
+ mainModule = Namespace.toModule namespace,
+ wrapper = Nothing,
+ sysdeps = psys <> Set.singleton "rustc",
+ out = pout <|> defaultOut,
+ compiler = Rustc,
+ compilerFlags = case pout of
+ Just o ->
+ map
+ Text.pack
+ [ "$CODEROOT" </> path,
+ "-o",
+ o
+ ]
+ Nothing -> panic "can't build rust libs",
+ builder = "base",
+ outPath = outToPath pout,
+ -- implement detectRustImports
+ srcs = Set.empty,
+ rundeps = prun,
+ hsGraph = Nothing,
+ ..
+ }
+ |> Just
+ |> pure
detectHaskellImports :: Analysis -> [Text] -> IO (Set Meta.Dep, Set FilePath)
detectHaskellImports _ contentLines = do
@@ -1219,6 +1209,213 @@ build andTest loud jobs cpus analysis = do
LogC.updateLineState namespace (isSuccess (fst result) ?: (LogC.Success, LogC.Failed))
pure result
+-- | Pipeline state machine for each target
+data TargetState
+ = TSQueued
+ | TSAnalyzing
+ | TSAnalysisFailed
+ | TSWaitingForDeps Target (Set Namespace)
+ | TSReadyToBuild Target
+ | TSBuilding Target
+ | TSComplete Target Exit.ExitCode
+
+-- | Coordinator manages the pipelined analyze→build flow
+data Coordinator = Coordinator
+ { coStates :: TVar (Map Namespace TargetState),
+ coAnalyzeQ :: TQueue Namespace,
+ coBuildQ :: TQueue Namespace,
+ coAllTargets :: Set Namespace,
+ coResults :: TVar [Exit.ExitCode],
+ coRemaining :: TVar Int,
+ coRoot :: FilePath
+ }
+
+initCoordinator :: FilePath -> [Namespace] -> IO Coordinator
+initCoordinator root nss =
+ atomically <| do
+ let allTargets = Set.fromList nss
+ states <- newTVar (Map.fromList [(ns, TSQueued) | ns <- nss])
+ analyzeQ <- newTQueue
+ buildQ <- newTQueue
+ results <- newTVar []
+ remaining <- newTVar (length nss)
+ forM_ nss (writeTQueue analyzeQ)
+ pure
+ Coordinator
+ { coStates = states,
+ coAnalyzeQ = analyzeQ,
+ coBuildQ = buildQ,
+ coAllTargets = allTargets,
+ coResults = results,
+ coRemaining = remaining,
+ coRoot = root
+ }
+
+computeDeps :: Coordinator -> Target -> Set Namespace
+computeDeps Coordinator {..} Target {..} =
+ let toNs path = Namespace.fromPath coRoot (coRoot </> path)
+ result =
+ srcs
+ |> Set.toList
+ |> map toNs
+ |> catMaybes
+ |> Set.fromList
+ |> flip Set.intersection coAllTargets
+ |> Set.delete namespace
+ in result
+
+tsIsComplete :: TargetState -> Bool
+tsIsComplete (TSComplete _ _) = True
+tsIsComplete _ = False
+
+pipelineAnalysisWorker :: Coordinator -> (Namespace -> IO (Maybe Target)) -> IO ()
+pipelineAnalysisWorker coord@Coordinator {..} analyzeFn = loop
+ where
+ loop = do
+ remaining <- readTVarIO coRemaining
+ when (remaining > 0) <| do
+ mNs <- atomically (tryReadTQueue coAnalyzeQ)
+ case mNs of
+ Nothing -> threadDelay 1000 >> loop
+ Just ns -> do
+ atomically <| modifyTVar' coStates (Map.insert ns TSAnalyzing)
+ LogC.updateLineState ns LogC.Analyzing
+ result <- analyzeFn ns
+ case result of
+ Nothing -> do
+ atomically <| do
+ modifyTVar' coStates (Map.insert ns TSAnalysisFailed)
+ modifyTVar' coRemaining (subtract 1)
+ LogC.updateLineState ns LogC.Failed
+ Just target -> do
+ let deps = computeDeps coord target
+ atomically <| do
+ states <- readTVar coStates
+ let pendingDeps = Set.filter (\d -> maybe True (tsIsComplete .> not) (Map.lookup d states)) deps
+ if Set.null pendingDeps
+ then do
+ modifyTVar' coStates (Map.insert ns (TSReadyToBuild target))
+ writeTQueue coBuildQ ns
+ else modifyTVar' coStates (Map.insert ns (TSWaitingForDeps target pendingDeps))
+ loop
+
+pipelineBuildWorker :: Bool -> Bool -> Int -> Int -> Coordinator -> IO ()
+pipelineBuildWorker andTest loud jobs cpus coord@Coordinator {..} = loop
+ where
+ loop = do
+ remaining <- readTVarIO coRemaining
+ when (remaining > 0) <| do
+ mNs <- atomically (tryReadTQueue coBuildQ)
+ case mNs of
+ Nothing -> threadDelay 1000 >> loop
+ Just ns -> do
+ mTarget <-
+ atomically <| do
+ states <- readTVar coStates
+ case Map.lookup ns states of
+ Just (TSReadyToBuild t) -> do
+ modifyTVar' coStates (Map.insert ns (TSBuilding t))
+ pure (Just t)
+ _ -> pure Nothing
+ case mTarget of
+ Nothing -> loop
+ Just target -> do
+ LogC.updateLineState ns LogC.Building
+ exitCode <- pipelineBuildOne andTest loud jobs cpus target
+ atomically <| do
+ modifyTVar' coStates (Map.insert ns (TSComplete target exitCode))
+ modifyTVar' coResults (exitCode :)
+ modifyTVar' coRemaining (subtract 1)
+ promoteWaiters coord ns
+ LogC.updateLineState ns (isSuccess exitCode ?: (LogC.Success, LogC.Failed))
+ loop
+
+promoteWaiters :: Coordinator -> Namespace -> STM ()
+promoteWaiters Coordinator {..} completedNs = do
+ states <- readTVar coStates
+ forM_ (Map.toList states) <| \(ns, st) ->
+ case st of
+ TSWaitingForDeps target deps -> do
+ let deps' = Set.delete completedNs deps
+ if Set.null deps'
+ then do
+ modifyTVar' coStates (Map.insert ns (TSReadyToBuild target))
+ writeTQueue coBuildQ ns
+ else modifyTVar' coStates (Map.insert ns (TSWaitingForDeps target deps'))
+ _ -> pure ()
+
+pipelineBuildOne :: Bool -> Bool -> Int -> Int -> Target -> IO Exit.ExitCode
+pipelineBuildOne andTest loud jobs cpus target@Target {..} = do
+ root <- getCoderoot
+ result <- case compiler of
+ CPython -> case out of
+ Just _ ->
+ nixBuild loud jobs cpus target
+ +> (\r -> (isSuccess (fst r) && andTest) ?: (test loud target, pure r))
+ Nothing ->
+ pure (Exit.ExitSuccess, mempty)
+ Gcc ->
+ nixBuild loud jobs cpus target
+ Ghc -> case out of
+ Nothing -> pure (Exit.ExitSuccess, mempty)
+ Just _ -> do
+ r <- nixBuild loud jobs cpus target
+ if andTest && (isSuccess <| fst r)
+ then test loud target
+ else pure r
+ Guile -> do
+ _ <- proc loud namespace (toNixFlag compiler) compilerFlags
+ case wrapper of
+ Nothing -> pure (Exit.ExitSuccess, mempty)
+ Just content -> do
+ writeFile (root </> outToPath out) content
+ p <- Dir.getPermissions <| root </> outToPath out
+ Dir.setPermissions (root </> outToPath out) (Dir.setOwnerExecutable True p)
+ pure (Exit.ExitSuccess, mempty)
+ NixBuild ->
+ Dir.getPermissions quapath /> Dir.executable +> \isExe ->
+ isExe
+ ?: ( proc
+ loud
+ namespace
+ (toNixFlag compiler)
+ ( compilerFlags
+ ++ [ "--max-jobs",
+ Text.pack <| str jobs,
+ "--cores",
+ Text.pack <| str cpus
+ ]
+ ),
+ Log.warn ["bild", "nix", nschunk namespace, "x bit not set, not building"]
+ >> pure (Exit.ExitSuccess, mempty)
+ )
+ Copy -> do
+ Log.warn ["bild", "copy", "not implemented yet", nschunk namespace]
+ pure (Exit.ExitSuccess, mempty)
+ Rustc ->
+ nixBuild loud jobs cpus target
+ Sbcl ->
+ proc loud namespace (toNixFlag compiler) compilerFlags
+ pure (fst result)
+
+pipelineBuild :: Bool -> Bool -> Int -> Int -> Int -> [Namespace] -> (Namespace -> IO (Maybe Target)) -> IO [Exit.ExitCode]
+pipelineBuild andTest loud analysisWorkers buildWorkers cpus namespaces analyzeFn = do
+ root <- getCoderoot
+ coord <- initCoordinator root namespaces
+ let spawnAnalysis = replicateM analysisWorkers (Async.async (pipelineAnalysisWorker coord analyzeFn))
+ let spawnBuild = replicateM buildWorkers (Async.async (pipelineBuildWorker andTest loud buildWorkers cpus coord))
+ threads <- (<>) </ spawnAnalysis <*> spawnBuild
+ let waitLoop = do
+ remaining <- readTVarIO (coRemaining coord)
+ if remaining == 0
+ then pure ()
+ else do
+ threadDelay 10000
+ waitLoop
+ waitLoop
+ traverse_ Async.cancel threads
+ readTVarIO (coResults coord)
+
data Proc = Proc
{ loud :: Bool,
cmd :: String,
diff --git a/Omni/Log/Concurrent.hs b/Omni/Log/Concurrent.hs
index 6dc7297..77131ef 100644
--- a/Omni/Log/Concurrent.hs
+++ b/Omni/Log/Concurrent.hs
@@ -20,13 +20,13 @@ import qualified Data.Text as Text
import Omni.Log.Terminal (OutputMode (..), TerminalInfo (..), detectTerminal, truncateToWidth)
import Omni.Namespace (Namespace)
import qualified Omni.Namespace as Namespace
-import Rainbow (chunk, fore, green, red, white)
+import Rainbow (chunk, fore, green, red, white, yellow)
import qualified Rainbow
import qualified System.Console.ANSI as ANSI
import qualified System.IO as IO
import System.IO.Unsafe (unsafePerformIO)
-data BuildState = Analyzing | Pending | Building | Success | Failed
+data BuildState = Analyzing | Pending | Building | Success | Failed | Skipped
deriving (Eq, Show)
data LineManager = LineManager
@@ -88,7 +88,6 @@ withLineManager nss action = do
MultiLine -> do
-- Multi-line mode: reserve lines for each namespace
let numLines = min (length nss) (tiHeight termInfo - 2)
- IO.hPutStrLn IO.stderr ""
replicateM_ numLines (IO.hPutStrLn IO.stderr "")
withMVar terminalLock <| \_ -> ANSI.hCursorUp IO.stderr numLines
@@ -196,6 +195,7 @@ updateLineState ns buildState = do
let (symbol, color) = case buildState of
Success -> ("✓", green)
Failed -> ("x", red)
+ Skipped -> ("_", yellow)
Analyzing -> ("+", white)
Pending -> (".", white)
Building -> ("~", white)
@@ -211,6 +211,9 @@ updateLineState ns buildState = do
Failed -> do
IO.hPutStrLn IO.stderr "" -- Keep failures visible
writeIORef lastOutputTransient False
+ Skipped -> do
+ IO.hPutStrLn IO.stderr "" -- Keep skipped visible
+ writeIORef lastOutputTransient False
_ -> writeIORef lastOutputTransient True -- Transient states overwrite
IO.hFlush IO.stderr
MultiLine ->
@@ -229,6 +232,7 @@ updateLineState ns buildState = do
let (symbol, colorFn) = case buildState of
Success -> ("✓", fore green)
Failed -> ("x", fore red)
+ Skipped -> ("_", fore yellow)
Analyzing -> ("+", identity)
Pending -> (".", identity)
Building -> ("~", identity)
diff --git a/Omni/Log/Terminal.hs b/Omni/Log/Terminal.hs
index 6832d17..1a4c717 100644
--- a/Omni/Log/Terminal.hs
+++ b/Omni/Log/Terminal.hs
@@ -54,10 +54,9 @@ detectTerminal = do
Just (h, w) -> (w, h)
Nothing -> (80, 24) -- sensible default
- -- Determine mode based on terminal width
+ -- Determine mode based on ANSI support
let mode
| not supportsANSI = SingleLine -- Fallback to single line for dumb terminals
- | width < 80 = SingleLine
| otherwise = MultiLine
pure