p2pHttpApp with a map of UUIDs to server states

This is early groundwork for making p2phttp support serving multiple
repositories from a single daemon.

So far only 1 repository is served still. And this commit breaks support
for proxying!
This commit is contained in:
Joey Hess 2024-11-20 12:51:25 -04:00
parent b8a717a617
commit 254073569f
No known key found for this signature in database
GPG key ID: DB12DB0FF05F8F38
3 changed files with 85 additions and 76 deletions

View file

@ -16,6 +16,7 @@ import P2P.Http.Server
import P2P.Http.Url import P2P.Http.Url
import qualified P2P.Protocol as P2P import qualified P2P.Protocol as P2P
import Utility.Env import Utility.Env
import Annex.UUID
import Servant import Servant
import qualified Network.Wai.Handler.Warp as Warp import qualified Network.Wai.Handler.Warp as Warp
@ -102,24 +103,27 @@ optParser _ = Options
)) ))
seek :: Options -> CommandSeek seek :: Options -> CommandSeek
seek o = getAnnexWorkerPool $ \workerpool -> seek o = do
withP2PConnections workerpool u <- getUUID
(fromMaybe 1 $ proxyConnectionsOption o) getAnnexWorkerPool $ \workerpool ->
(fromMaybe 1 $ clusterJobsOption o) withP2PConnections workerpool
(go workerpool) (fromMaybe 1 $ proxyConnectionsOption o)
(fromMaybe 1 $ clusterJobsOption o)
(go u workerpool)
where where
go workerpool acquireconn = liftIO $ do go u workerpool acquireconn = liftIO $ do
authenv <- getAuthEnv authenv <- getAuthEnv
st <- mkP2PHttpServerState acquireconn workerpool $ st <- mkP2PHttpServerState acquireconn workerpool $
mkGetServerMode authenv o mkGetServerMode authenv o
let mst = M.singleton u st
let settings = Warp.setPort port $ Warp.setHost host $ let settings = Warp.setPort port $ Warp.setHost host $
Warp.defaultSettings Warp.defaultSettings
case (certFileOption o, privateKeyFileOption o) of case (certFileOption o, privateKeyFileOption o) of
(Nothing, Nothing) -> Warp.runSettings settings (p2pHttpApp st) (Nothing, Nothing) -> Warp.runSettings settings (p2pHttpApp mst)
(Just certfile, Just privatekeyfile) -> do (Just certfile, Just privatekeyfile) -> do
let tlssettings = Warp.tlsSettingsChain let tlssettings = Warp.tlsSettingsChain
certfile (chainFileOption o) privatekeyfile certfile (chainFileOption o) privatekeyfile
Warp.runTLS tlssettings settings (p2pHttpApp st) Warp.runTLS tlssettings settings (p2pHttpApp mst)
_ -> giveup "You must use both --certfile and --privatekeyfile options to enable HTTPS." _ -> giveup "You must use both --certfile and --privatekeyfile options to enable HTTPS."
port = maybe port = maybe

View file

@ -40,15 +40,16 @@ import qualified Servant.Types.SourceT as S
import qualified Data.ByteString as B import qualified Data.ByteString as B
import qualified Data.ByteString.Lazy as L import qualified Data.ByteString.Lazy as L
import qualified Data.ByteString.Lazy.Internal as LI import qualified Data.ByteString.Lazy.Internal as LI
import qualified Data.Map as M
import Control.Concurrent.Async import Control.Concurrent.Async
import Control.Concurrent import Control.Concurrent
import System.IO.Unsafe import System.IO.Unsafe
import Data.Either import Data.Either
p2pHttpApp :: P2PHttpServerState -> Application p2pHttpApp :: M.Map UUID P2PHttpServerState -> Application
p2pHttpApp = serve p2pHttpAPI . serveP2pHttp p2pHttpApp = serve p2pHttpAPI . serveP2pHttp
serveP2pHttp :: P2PHttpServerState -> Server P2PHttpAPI serveP2pHttp :: M.Map UUID P2PHttpServerState -> Server P2PHttpAPI
serveP2pHttp st serveP2pHttp st
= serveGet st = serveGet st
:<|> serveGet st :<|> serveGet st
@ -91,7 +92,7 @@ serveP2pHttp st
:<|> serveGetGeneric st :<|> serveGetGeneric st
serveGetGeneric serveGetGeneric
:: P2PHttpServerState :: M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> B64Key -> B64Key
-> Maybe (B64UUID ClientSide) -> Maybe (B64UUID ClientSide)
@ -109,7 +110,7 @@ serveGetGeneric st su@(B64UUID u) k mcu bypass =
serveGet serveGet
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> B64Key -> B64Key
@ -120,8 +121,8 @@ serveGet
-> IsSecure -> IsSecure
-> Maybe Auth -> Maybe Auth
-> Handler (Headers '[DataLengthHeader] (S.SourceT IO B.ByteString)) -> Handler (Headers '[DataLengthHeader] (S.SourceT IO B.ByteString))
serveGet st su apiver (B64Key k) cu bypass baf startat sec auth = do serveGet mst su apiver (B64Key k) cu bypass baf startat sec auth = do
conn <- getP2PConnection apiver st cu su bypass sec auth ReadAction id (conn, st) <- getP2PConnection apiver mst cu su bypass sec auth ReadAction id
bsv <- liftIO newEmptyTMVarIO bsv <- liftIO newEmptyTMVarIO
endv <- liftIO newEmptyTMVarIO endv <- liftIO newEmptyTMVarIO
validityv <- liftIO newEmptyTMVarIO validityv <- liftIO newEmptyTMVarIO
@ -222,7 +223,7 @@ serveGet st su apiver (B64Key k) cu bypass baf startat sec auth = do
serveCheckPresent serveCheckPresent
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> B64Key -> B64Key
@ -233,14 +234,14 @@ serveCheckPresent
-> Handler CheckPresentResult -> Handler CheckPresentResult
serveCheckPresent st su apiver (B64Key k) cu bypass sec auth = do serveCheckPresent st su apiver (B64Key k) cu bypass sec auth = do
res <- withP2PConnection apiver st cu su bypass sec auth ReadAction id res <- withP2PConnection apiver st cu su bypass sec auth ReadAction id
$ \conn -> liftIO $ proxyClientNetProto conn $ checkPresent k $ \(conn, _) -> liftIO $ proxyClientNetProto conn $ checkPresent k
case res of case res of
Right b -> return (CheckPresentResult b) Right b -> return (CheckPresentResult b)
Left err -> throwError $ err500 { errBody = encodeBL err } Left err -> throwError $ err500 { errBody = encodeBL err }
serveRemove serveRemove
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> (RemoveResultPlus -> t) -> (RemoveResultPlus -> t)
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
@ -252,7 +253,7 @@ serveRemove
-> Handler t -> Handler t
serveRemove st resultmangle su apiver (B64Key k) cu bypass sec auth = do serveRemove st resultmangle su apiver (B64Key k) cu bypass sec auth = do
res <- withP2PConnection apiver st cu su bypass sec auth RemoveAction id res <- withP2PConnection apiver st cu su bypass sec auth RemoveAction id
$ \conn -> $ \(conn, _) ->
liftIO $ proxyClientNetProto conn $ remove Nothing k liftIO $ proxyClientNetProto conn $ remove Nothing k
case res of case res of
(Right b, plusuuids) -> return $ resultmangle $ (Right b, plusuuids) -> return $ resultmangle $
@ -262,7 +263,7 @@ serveRemove st resultmangle su apiver (B64Key k) cu bypass sec auth = do
serveRemoveBefore serveRemoveBefore
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> B64Key -> B64Key
@ -274,7 +275,7 @@ serveRemoveBefore
-> Handler RemoveResultPlus -> Handler RemoveResultPlus
serveRemoveBefore st su apiver (B64Key k) cu bypass (Timestamp ts) sec auth = do serveRemoveBefore st su apiver (B64Key k) cu bypass (Timestamp ts) sec auth = do
res <- withP2PConnection apiver st cu su bypass sec auth RemoveAction id res <- withP2PConnection apiver st cu su bypass sec auth RemoveAction id
$ \conn -> $ \(conn, _) ->
liftIO $ proxyClientNetProto conn $ liftIO $ proxyClientNetProto conn $
removeBeforeRemoteEndTime ts k removeBeforeRemoteEndTime ts k
case res of case res of
@ -285,7 +286,7 @@ serveRemoveBefore st su apiver (B64Key k) cu bypass (Timestamp ts) sec auth = do
serveGetTimestamp serveGetTimestamp
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> B64UUID ClientSide -> B64UUID ClientSide
@ -295,7 +296,7 @@ serveGetTimestamp
-> Handler GetTimestampResult -> Handler GetTimestampResult
serveGetTimestamp st su apiver cu bypass sec auth = do serveGetTimestamp st su apiver cu bypass sec auth = do
res <- withP2PConnection apiver st cu su bypass sec auth ReadAction id res <- withP2PConnection apiver st cu su bypass sec auth ReadAction id
$ \conn -> $ \(conn, _) ->
liftIO $ proxyClientNetProto conn getTimestamp liftIO $ proxyClientNetProto conn getTimestamp
case res of case res of
Right ts -> return $ GetTimestampResult (Timestamp ts) Right ts -> return $ GetTimestampResult (Timestamp ts)
@ -304,7 +305,7 @@ serveGetTimestamp st su apiver cu bypass sec auth = do
servePut servePut
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> (PutResultPlus -> t) -> (PutResultPlus -> t)
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
@ -319,28 +320,28 @@ servePut
-> IsSecure -> IsSecure
-> Maybe Auth -> Maybe Auth
-> Handler t -> Handler t
servePut st resultmangle su apiver (Just True) _ k cu bypass baf _ _ sec auth = do servePut mst resultmangle su apiver (Just True) _ k cu bypass baf _ _ sec auth = do
res <- withP2PConnection' apiver st cu su bypass sec auth WriteAction res <- withP2PConnection' apiver mst cu su bypass sec auth WriteAction
(\cst -> cst { connectionWaitVar = False }) (liftIO . protoaction) (\cst -> cst { connectionWaitVar = False }) (liftIO . protoaction)
servePutResult resultmangle res servePutResult resultmangle res
where where
protoaction conn = servePutAction st conn k baf $ \_offset -> do protoaction conn = servePutAction conn k baf $ \_offset -> do
net $ sendMessage DATA_PRESENT net $ sendMessage DATA_PRESENT
checkSuccessPlus checkSuccessPlus
servePut st resultmangle su apiver _datapresent (DataLength len) k cu bypass baf moffset stream sec auth = do servePut mst resultmangle su apiver _datapresent (DataLength len) k cu bypass baf moffset stream sec auth = do
validityv <- liftIO newEmptyTMVarIO validityv <- liftIO newEmptyTMVarIO
let validitycheck = local $ runValidityCheck $ let validitycheck = local $ runValidityCheck $
liftIO $ atomically $ readTMVar validityv liftIO $ atomically $ readTMVar validityv
tooshortv <- liftIO newEmptyTMVarIO tooshortv <- liftIO newEmptyTMVarIO
content <- liftIO $ S.unSourceT stream (gather validityv tooshortv) content <- liftIO $ S.unSourceT stream (gather validityv tooshortv)
res <- withP2PConnection' apiver st cu su bypass sec auth WriteAction res <- withP2PConnection' apiver mst cu su bypass sec auth WriteAction
(\cst -> cst { connectionWaitVar = False }) $ \conn -> do (\cst -> cst { connectionWaitVar = False }) $ \(conn, st) -> do
liftIO $ void $ async $ checktooshort conn tooshortv liftIO $ void $ async $ checktooshort conn tooshortv
liftIO (protoaction conn content validitycheck) liftIO (protoaction conn st content validitycheck)
servePutResult resultmangle res servePutResult resultmangle res
where where
protoaction conn content validitycheck = protoaction conn st content validitycheck =
servePutAction st conn k baf $ \offset' -> servePutAction (conn, st) k baf $ \offset' ->
let offsetdelta = offset' - offset let offsetdelta = offset' - offset
in case compare offset' offset of in case compare offset' offset of
EQ -> sendContent' nullMeterUpdate (Len len) EQ -> sendContent' nullMeterUpdate (Len len)
@ -396,13 +397,12 @@ servePut st resultmangle su apiver _datapresent (DataLength len) k cu bypass baf
closeP2PConnection conn closeP2PConnection conn
servePutAction servePutAction
:: P2PHttpServerState :: (P2PConnectionPair, P2PHttpServerState)
-> P2PConnectionPair
-> B64Key -> B64Key
-> Maybe B64FilePath -> Maybe B64FilePath
-> (P2P.Protocol.Offset -> Proto (Maybe [UUID])) -> (P2P.Protocol.Offset -> Proto (Maybe [UUID]))
-> IO (Either SomeException (Either ProtoFailure (Maybe [UUID]))) -> IO (Either SomeException (Either ProtoFailure (Maybe [UUID])))
servePutAction st conn (B64Key k) baf a = inAnnexWorker st $ servePutAction (conn, st) (B64Key k) baf a = inAnnexWorker st $
enteringStage (TransferStage Download) $ enteringStage (TransferStage Download) $
runFullProto (clientRunState conn) (clientP2PConnection conn) $ runFullProto (clientRunState conn) (clientP2PConnection conn) $
put' k af a put' k af a
@ -422,7 +422,7 @@ servePutResult resultmangle res = case res of
servePut' servePut'
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> (PutResultPlus -> t) -> (PutResultPlus -> t)
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
@ -440,7 +440,7 @@ servePut' st resultmangle su v = servePut st resultmangle su v Nothing
servePutOffset servePutOffset
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> (PutOffsetResultPlus -> t) -> (PutOffsetResultPlus -> t)
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
@ -452,7 +452,7 @@ servePutOffset
-> Handler t -> Handler t
servePutOffset st resultmangle su apiver (B64Key k) cu bypass sec auth = do servePutOffset st resultmangle su apiver (B64Key k) cu bypass sec auth = do
res <- withP2PConnection apiver st cu su bypass sec auth WriteAction res <- withP2PConnection apiver st cu su bypass sec auth WriteAction
(\cst -> cst { connectionWaitVar = False }) $ \conn -> (\cst -> cst { connectionWaitVar = False }) $ \(conn, _) ->
liftIO $ proxyClientNetProto conn $ getPutOffset k af liftIO $ proxyClientNetProto conn $ getPutOffset k af
case res of case res of
Right offset -> return $ resultmangle $ Right offset -> return $ resultmangle $
@ -464,7 +464,7 @@ servePutOffset st resultmangle su apiver (B64Key k) cu bypass sec auth = do
serveLockContent serveLockContent
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> B64Key -> B64Key
@ -473,8 +473,8 @@ serveLockContent
-> IsSecure -> IsSecure
-> Maybe Auth -> Maybe Auth
-> Handler LockResult -> Handler LockResult
serveLockContent st su apiver (B64Key k) cu bypass sec auth = do serveLockContent mst su apiver (B64Key k) cu bypass sec auth = do
conn <- getP2PConnection apiver st cu su bypass sec auth LockAction id (conn, st) <- getP2PConnection apiver mst cu su bypass sec auth LockAction id
let lock = do let lock = do
lockresv <- newEmptyTMVarIO lockresv <- newEmptyTMVarIO
unlockv <- newEmptyTMVarIO unlockv <- newEmptyTMVarIO
@ -501,7 +501,7 @@ serveLockContent st su apiver (B64Key k) cu bypass sec auth = do
serveKeepLocked serveKeepLocked
:: APIVersion v :: APIVersion v
=> P2PHttpServerState => M.Map UUID P2PHttpServerState
-> B64UUID ServerSide -> B64UUID ServerSide
-> v -> v
-> LockID -> LockID
@ -513,15 +513,15 @@ serveKeepLocked
-> Maybe KeepAlive -> Maybe KeepAlive
-> S.SourceT IO UnlockRequest -> S.SourceT IO UnlockRequest
-> Handler LockResult -> Handler LockResult
serveKeepLocked st _su _apiver lckid _cu _bypass sec auth _ _ unlockrequeststream = do serveKeepLocked mst su _apiver lckid _cu _bypass sec auth _ _ unlockrequeststream = do
checkAuthActionClass st sec auth LockAction $ \_ -> do checkAuthActionClass mst su sec auth LockAction $ \st _ -> do
liftIO $ keepingLocked lckid st liftIO $ keepingLocked lckid st
_ <- liftIO $ S.unSourceT unlockrequeststream go _ <- liftIO $ S.unSourceT unlockrequeststream (go st)
return (LockResult False Nothing) return (LockResult False Nothing)
where where
go S.Stop = dropLock lckid st go st S.Stop = dropLock lckid st
go (S.Error _err) = dropLock lckid st go st (S.Error _err) = dropLock lckid st
go (S.Skip s) = go s go st (S.Skip s) = go st s
go (S.Effect ms) = ms >>= go go st (S.Effect ms) = ms >>= go st
go (S.Yield (UnlockRequest False) s) = go s go st (S.Yield (UnlockRequest False) s) = go st s
go (S.Yield (UnlockRequest True) _) = dropLock lckid st go st (S.Yield (UnlockRequest True) _) = dropLock lckid st

View file

@ -75,7 +75,7 @@ data ActionClass = ReadAction | WriteAction | RemoveAction | LockAction
withP2PConnection withP2PConnection
:: APIVersion v :: APIVersion v
=> v => v
-> P2PHttpServerState -> M.Map UUID P2PHttpServerState
-> B64UUID ClientSide -> B64UUID ClientSide
-> B64UUID ServerSide -> B64UUID ServerSide
-> [B64UUID Bypass] -> [B64UUID Bypass]
@ -83,10 +83,10 @@ withP2PConnection
-> Maybe Auth -> Maybe Auth
-> ActionClass -> ActionClass
-> (ConnectionParams -> ConnectionParams) -> (ConnectionParams -> ConnectionParams)
-> (P2PConnectionPair -> Handler (Either ProtoFailure a)) -> ((P2PConnectionPair, P2PHttpServerState) -> Handler (Either ProtoFailure a))
-> Handler a -> Handler a
withP2PConnection apiver st cu su bypass sec auth actionclass fconnparams connaction = withP2PConnection apiver mst cu su bypass sec auth actionclass fconnparams connaction =
withP2PConnection' apiver st cu su bypass sec auth actionclass fconnparams connaction' withP2PConnection' apiver mst cu su bypass sec auth actionclass fconnparams connaction'
where where
connaction' conn = connaction conn >>= \case connaction' conn = connaction conn >>= \case
Right r -> return r Right r -> return r
@ -96,7 +96,7 @@ withP2PConnection apiver st cu su bypass sec auth actionclass fconnparams connac
withP2PConnection' withP2PConnection'
:: APIVersion v :: APIVersion v
=> v => v
-> P2PHttpServerState -> M.Map UUID P2PHttpServerState
-> B64UUID ClientSide -> B64UUID ClientSide
-> B64UUID ServerSide -> B64UUID ServerSide
-> [B64UUID Bypass] -> [B64UUID Bypass]
@ -104,17 +104,17 @@ withP2PConnection'
-> Maybe Auth -> Maybe Auth
-> ActionClass -> ActionClass
-> (ConnectionParams -> ConnectionParams) -> (ConnectionParams -> ConnectionParams)
-> (P2PConnectionPair -> Handler a) -> ((P2PConnectionPair, P2PHttpServerState) -> Handler a)
-> Handler a -> Handler a
withP2PConnection' apiver st cu su bypass sec auth actionclass fconnparams connaction = do withP2PConnection' apiver mst cu su bypass sec auth actionclass fconnparams connaction = do
conn <- getP2PConnection apiver st cu su bypass sec auth actionclass fconnparams (conn, st) <- getP2PConnection apiver mst cu su bypass sec auth actionclass fconnparams
connaction conn connaction (conn, st)
`finally` liftIO (releaseP2PConnection conn) `finally` liftIO (releaseP2PConnection conn)
getP2PConnection getP2PConnection
:: APIVersion v :: APIVersion v
=> v => v
-> P2PHttpServerState -> M.Map UUID P2PHttpServerState
-> B64UUID ClientSide -> B64UUID ClientSide
-> B64UUID ServerSide -> B64UUID ServerSide
-> [B64UUID Bypass] -> [B64UUID Bypass]
@ -122,16 +122,16 @@ getP2PConnection
-> Maybe Auth -> Maybe Auth
-> ActionClass -> ActionClass
-> (ConnectionParams -> ConnectionParams) -> (ConnectionParams -> ConnectionParams)
-> Handler P2PConnectionPair -> Handler (P2PConnectionPair, P2PHttpServerState)
getP2PConnection apiver st cu su bypass sec auth actionclass fconnparams = getP2PConnection apiver mst cu su bypass sec auth actionclass fconnparams =
checkAuthActionClass st sec auth actionclass go checkAuthActionClass mst su sec auth actionclass go
where where
go servermode = liftIO (acquireP2PConnection st cp) >>= \case go st servermode = liftIO (acquireP2PConnection st cp) >>= \case
Left (ConnectionFailed err) -> Left (ConnectionFailed err) ->
throwError err502 { errBody = encodeBL err } throwError err502 { errBody = encodeBL err }
Left TooManyConnections -> Left TooManyConnections ->
throwError err503 throwError err503
Right v -> return v Right v -> return (v, st)
where where
cp = fconnparams $ ConnectionParams cp = fconnparams $ ConnectionParams
{ connectionProtocolVersion = protocolVersion apiver { connectionProtocolVersion = protocolVersion apiver
@ -143,29 +143,34 @@ getP2PConnection apiver st cu su bypass sec auth actionclass fconnparams =
} }
checkAuthActionClass checkAuthActionClass
:: P2PHttpServerState :: M.Map UUID P2PHttpServerState
-> B64UUID ServerSide
-> IsSecure -> IsSecure
-> Maybe Auth -> Maybe Auth
-> ActionClass -> ActionClass
-> (P2P.ServerMode -> Handler a) -> (P2PHttpServerState -> P2P.ServerMode -> Handler a)
-> Handler a -> Handler a
checkAuthActionClass st sec auth actionclass go = checkAuthActionClass mst su sec auth actionclass go =
case (sm, actionclass) of case M.lookup (fromB64UUID su) mst of
Just st -> select st
Nothing -> throwError err404
where
select st = case (sm, actionclass) of
(ServerMode { serverMode = P2P.ServeReadWrite }, _) -> (ServerMode { serverMode = P2P.ServeReadWrite }, _) ->
go P2P.ServeReadWrite go st P2P.ServeReadWrite
(ServerMode { unauthenticatedLockingAllowed = True }, LockAction) -> (ServerMode { unauthenticatedLockingAllowed = True }, LockAction) ->
go P2P.ServeReadOnly go st P2P.ServeReadOnly
(ServerMode { serverMode = P2P.ServeAppendOnly }, RemoveAction) -> (ServerMode { serverMode = P2P.ServeAppendOnly }, RemoveAction) ->
throwError $ forbiddenWithoutAuth sm throwError $ forbiddenWithoutAuth sm
(ServerMode { serverMode = P2P.ServeAppendOnly }, _) -> (ServerMode { serverMode = P2P.ServeAppendOnly }, _) ->
go P2P.ServeAppendOnly go st P2P.ServeAppendOnly
(ServerMode { serverMode = P2P.ServeReadOnly }, ReadAction) -> (ServerMode { serverMode = P2P.ServeReadOnly }, ReadAction) ->
go P2P.ServeReadOnly go st P2P.ServeReadOnly
(ServerMode { serverMode = P2P.ServeReadOnly }, _) -> (ServerMode { serverMode = P2P.ServeReadOnly }, _) ->
throwError $ forbiddenWithoutAuth sm throwError $ forbiddenWithoutAuth sm
(CannotServeRequests, _) -> throwError basicAuthRequired (CannotServeRequests, _) -> throwError basicAuthRequired
where where
sm = getServerMode st sec auth sm = getServerMode st sec auth
forbiddenAction :: ServerError forbiddenAction :: ServerError
forbiddenAction = err403 forbiddenAction = err403