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

View file

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