{-# LANGUAGE DataKinds           #-}
{-# LANGUAGE FlexibleContexts    #-}
{-# LANGUAGE GADTs               #-}
{-# LANGUAGE NamedFieldPuns      #-}
{-# LANGUAGE PolyKinds           #-}
{-# LANGUAGE RankNTypes          #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications    #-}
{-# LANGUAGE TypeFamilies        #-}
{-# LANGUAGE TypeOperators       #-}

module Ouroboros.Network.Protocol.TxSubmission2.Codec
  ( codecTxSubmission2
  , anncodecTxSubmission2
  , codecTxSubmission2Id
  , byteLimitsTxSubmission2
  , timeLimitsTxSubmission2
  , WithBytes (..)
  ) where

import Control.Monad.Class.MonadST
import Control.Monad.Class.MonadTime.SI
import Data.ByteString.Lazy (ByteString)
import Data.ByteString.Lazy qualified as BSL
import Data.Constraint
import Data.Functor.Identity
import Data.Kind (Type)
import Data.List.NonEmpty qualified as NonEmpty
import Data.Type.Equality
import Text.Printf

import Codec.CBOR.Decoding qualified as CBOR
import Codec.CBOR.Encoding qualified as CBOR
import Codec.CBOR.Read qualified as CBOR

import Network.TypedProtocol.Codec.CBOR

import Ouroboros.Network.Protocol.Limits
import Ouroboros.Network.Protocol.TxSubmission2.Type

-- | Byte Limits.
byteLimitsTxSubmission2 :: forall bytes txid tx.
                           (bytes -> Word)
                        -> ProtocolSizeLimits (TxSubmission2 txid tx) bytes
byteLimitsTxSubmission2 :: forall bytes txid tx.
(bytes -> Word) -> ProtocolSizeLimits (TxSubmission2 txid tx) bytes
byteLimitsTxSubmission2 = (forall (st :: TxSubmission2 txid tx).
 ActiveState st =>
 StateToken st -> Word)
-> (bytes -> Word)
-> ProtocolSizeLimits (TxSubmission2 txid tx) bytes
forall ps bytes.
(forall (st :: ps). ActiveState st => StateToken st -> Word)
-> (bytes -> Word) -> ProtocolSizeLimits ps bytes
ProtocolSizeLimits StateToken st -> Word
forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st -> Word
stateToLimit
  where
    stateToLimit :: forall (st :: TxSubmission2 txid tx).
                    ActiveState st => StateToken st -> Word
    stateToLimit :: forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st -> Word
stateToLimit StateToken st
SingTxSubmission st
SingInit                    = Word
smallByteLimit
    stateToLimit (SingTxIds SingBlockingStyle stBlocking
SingBlocking)    = Word
largeByteLimit
    stateToLimit (SingTxIds SingBlockingStyle stBlocking
SingNonBlocking) = Word
largeByteLimit
    stateToLimit StateToken st
SingTxSubmission st
SingTxs                     = Word
largeByteLimit
    stateToLimit StateToken st
SingTxSubmission st
SingIdle                    = Word
smallByteLimit
    stateToLimit a :: StateToken st
a@StateToken st
SingTxSubmission st
SingDone                  = StateToken 'StDone -> forall a. a
forall ps (st :: ps).
(StateAgency st ~ 'NobodyAgency, ActiveState st) =>
StateToken st -> forall a. a
notActiveState StateToken st
StateToken 'StDone
a


-- | 'TxSubmission2' time limits.
--
-- +-----------------------------+---------------+
-- | 'TxSubmission2' state       | timeout (s)   |
-- +=============================+===============+
-- | `StInit`                    | `waitForever` |
-- +-----------------------------+---------------+
-- | `StIdle`                    | `waitForever` |
-- +-----------------------------+---------------+
-- | @'StTxIds' 'StBlocking'@    | `waitForever` |
-- +-----------------------------+---------------+
-- | @'StTxIds' 'StNonBlocking'@ | `shortWait`   |
-- +-----------------------------+---------------+
-- | `StTxs`                     | `shortWait`   |
-- +-----------------------------+---------------+
--
timeLimitsTxSubmission2 :: forall (txid :: Type) (tx :: Type). ProtocolTimeLimits (TxSubmission2 txid tx)
timeLimitsTxSubmission2 :: forall txid tx. ProtocolTimeLimits (TxSubmission2 txid tx)
timeLimitsTxSubmission2 = (forall (st :: TxSubmission2 txid tx).
 ActiveState st =>
 StateToken st -> Maybe DiffTime)
-> ProtocolTimeLimits (TxSubmission2 txid tx)
forall ps.
(forall (st :: ps).
 ActiveState st =>
 StateToken st -> Maybe DiffTime)
-> ProtocolTimeLimits ps
ProtocolTimeLimits StateToken st -> Maybe DiffTime
forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st -> Maybe DiffTime
stateToLimit
  where
    stateToLimit :: forall (st :: TxSubmission2 txid tx).
                    ActiveState st => StateToken st -> Maybe DiffTime
    stateToLimit :: forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st -> Maybe DiffTime
stateToLimit StateToken st
SingTxSubmission st
SingInit                    = Maybe DiffTime
waitForever
    stateToLimit (SingTxIds SingBlockingStyle stBlocking
SingBlocking)    = Maybe DiffTime
waitForever
    stateToLimit (SingTxIds SingBlockingStyle stBlocking
SingNonBlocking) = Maybe DiffTime
shortWait
    stateToLimit StateToken st
SingTxSubmission st
SingTxs                     = Maybe DiffTime
shortWait
    stateToLimit StateToken st
SingTxSubmission st
SingIdle                    = Maybe DiffTime
waitForever
    stateToLimit a :: StateToken st
a@StateToken st
SingTxSubmission st
SingDone                  = StateToken 'StDone -> forall a. a
forall ps (st :: ps).
(StateAgency st ~ 'NobodyAgency, ActiveState st) =>
StateToken st -> forall a. a
notActiveState StateToken st
StateToken 'StDone
a


data WithBytes a = WithBytes {
      forall a. WithBytes a -> ByteString
cborBytes   :: ByteString,
      -- ^ cbor encoding
      forall a. WithBytes a -> a
cborPayload :: a
      -- ^ decoded structure
    }
  deriving (Int -> WithBytes a -> ShowS
[WithBytes a] -> ShowS
WithBytes a -> String
(Int -> WithBytes a -> ShowS)
-> (WithBytes a -> String)
-> ([WithBytes a] -> ShowS)
-> Show (WithBytes a)
forall a. Show a => Int -> WithBytes a -> ShowS
forall a. Show a => [WithBytes a] -> ShowS
forall a. Show a => WithBytes a -> String
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall a. Show a => Int -> WithBytes a -> ShowS
showsPrec :: Int -> WithBytes a -> ShowS
$cshow :: forall a. Show a => WithBytes a -> String
show :: WithBytes a -> String
$cshowList :: forall a. Show a => [WithBytes a] -> ShowS
showList :: [WithBytes a] -> ShowS
Show, WithBytes a -> WithBytes a -> Bool
(WithBytes a -> WithBytes a -> Bool)
-> (WithBytes a -> WithBytes a -> Bool) -> Eq (WithBytes a)
forall a. Eq a => WithBytes a -> WithBytes a -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall a. Eq a => WithBytes a -> WithBytes a -> Bool
== :: WithBytes a -> WithBytes a -> Bool
$c/= :: forall a. Eq a => WithBytes a -> WithBytes a -> Bool
/= :: WithBytes a -> WithBytes a -> Bool
Eq)

encodeWithBytes :: WithBytes a -> CBOR.Encoding
encodeWithBytes :: forall a. WithBytes a -> Encoding
encodeWithBytes =
    -- this should be equivalent to
    -- `CBOR.encodePreEncoded . BSL.toStrict . cborBytes`
    -- but it doesn't copy the bytes
    (ByteString -> Encoding) -> [ByteString] -> Encoding
forall m a. Monoid m => (a -> m) -> [a] -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap ByteString -> Encoding
CBOR.encodePreEncoded ([ByteString] -> Encoding)
-> (WithBytes a -> [ByteString]) -> WithBytes a -> Encoding
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ByteString -> [ByteString]
BSL.toChunks (ByteString -> [ByteString])
-> (WithBytes a -> ByteString) -> WithBytes a -> [ByteString]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. WithBytes a -> ByteString
forall a. WithBytes a -> ByteString
cborBytes

-- | A bytespan functor.
--
newtype WithByteSpan a = WithByteSpan (a, CBOR.ByteOffset, CBOR.ByteOffset)

anncodecTxSubmission2
  :: forall (txid :: Type) (tx :: Type) m.
     MonadST m
  => (txid -> CBOR.Encoding)
  -- ^ encode 'txid'
  -> (forall s . CBOR.Decoder s txid)
  -- ^ decode 'txid'
  -> (forall s . CBOR.Decoder s tx)
  -- ^ decode transaction
  -> AnnotatedCodec (TxSubmission2 txid (WithBytes tx)) CBOR.DeserialiseFailure m ByteString
anncodecTxSubmission2 :: forall txid tx (m :: * -> *).
MonadST m =>
(txid -> Encoding)
-> (forall s. Decoder s txid)
-> (forall s. Decoder s tx)
-> AnnotatedCodec
     (TxSubmission2 txid (WithBytes tx)) DeserialiseFailure m ByteString
anncodecTxSubmission2 txid -> Encoding
encodeTxId forall s. Decoder s txid
decodeTxId
                                 forall s. Decoder s tx
decodeTx =
    (forall (st :: TxSubmission2 txid (WithBytes tx))
        (st' :: TxSubmission2 txid (WithBytes tx)).
 (StateTokenI st, ActiveState st) =>
 Message (TxSubmission2 txid (WithBytes tx)) st st' -> Encoding)
-> (forall (st :: TxSubmission2 txid (WithBytes tx)) s.
    ActiveState st =>
    StateToken st -> Decoder s (Annotator ByteString st))
-> CodecF
     (TxSubmission2 txid (WithBytes tx))
     DeserialiseFailure
     m
     (Annotator ByteString)
     ByteString
forall ps (m :: * -> *) (f :: ps -> *).
MonadST m =>
(forall (st :: ps) (st' :: ps).
 (StateTokenI st, ActiveState st) =>
 Message ps st st' -> Encoding)
-> (forall (st :: ps) s.
    ActiveState st =>
    StateToken st -> Decoder s (f st))
-> CodecF ps DeserialiseFailure m f ByteString
mkCodecCborLazyBS
      ((txid -> Encoding)
-> (WithBytes tx -> Encoding)
-> Message (TxSubmission2 txid (WithBytes tx)) st st'
-> Encoding
forall txid tx (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(txid -> Encoding)
-> (tx -> Encoding)
-> Message (TxSubmission2 txid tx) st st'
-> Encoding
encodeTxSubmission2 txid -> Encoding
encodeTxId WithBytes tx -> Encoding
forall a. WithBytes a -> Encoding
encodeWithBytes)
      StateToken st -> Decoder s (Annotator ByteString st)
StateToken st -> forall s. Decoder s (Annotator ByteString st)
forall (st :: TxSubmission2 txid (WithBytes tx)).
ActiveState st =>
StateToken st -> forall s. Decoder s (Annotator ByteString st)
forall (st :: TxSubmission2 txid (WithBytes tx)) s.
ActiveState st =>
StateToken st -> Decoder s (Annotator ByteString st)
decode
  where
    decode :: forall (st :: TxSubmission2 txid (WithBytes tx)).
              ActiveState st
           => StateToken st
           -> forall s. CBOR.Decoder s (Annotator ByteString st)
    decode :: forall (st :: TxSubmission2 txid (WithBytes tx)).
ActiveState st =>
StateToken st -> forall s. Decoder s (Annotator ByteString st)
decode =
      forall (withBytes :: * -> *) (withByteSpan :: * -> *) bytes txid tx
       (st :: TxSubmission2 txid (withBytes tx)) s.
ActiveState st =>
(bytes -> withByteSpan tx -> withBytes tx)
-> (forall a s'. Decoder s' a -> Decoder s' (withByteSpan a))
-> (forall s'. Decoder s' txid)
-> (forall s'. Decoder s' tx)
-> StateToken st
-> Decoder s (Annotator bytes st)
decodeTxSubmission2 @WithBytes
                          @WithByteSpan
                          @ByteString
                          ByteString -> WithByteSpan tx -> WithBytes tx
forall a. ByteString -> WithByteSpan a -> WithBytes a
mkWithBytes
                          Decoder s' a -> Decoder s' (WithByteSpan a)
forall s a. Decoder s a -> Decoder s (WithByteSpan a)
forall a s'. Decoder s' a -> Decoder s' (WithByteSpan a)
decodeWithByteSpan
                          Decoder s' txid
forall s. Decoder s txid
decodeTxId Decoder s' tx
forall s. Decoder s tx
decodeTx

    mkWithBytes
      :: ByteString
      -> WithByteSpan a
      -> WithBytes a
    mkWithBytes :: forall a. ByteString -> WithByteSpan a -> WithBytes a
mkWithBytes ByteString
bytes (WithByteSpan (a
cborPayload, Int64
start, Int64
end)) =
        WithBytes {
          cborBytes :: ByteString
cborBytes = Int64 -> ByteString -> ByteString
BSL.take (Int64
end Int64 -> Int64 -> Int64
forall a. Num a => a -> a -> a
- Int64
start) (ByteString -> ByteString) -> ByteString -> ByteString
forall a b. (a -> b) -> a -> b
$ Int64 -> ByteString -> ByteString
BSL.drop Int64
start ByteString
bytes,
          a
cborPayload :: a
cborPayload :: a
cborPayload
        }

    decodeWithByteSpan :: CBOR.Decoder s a -> CBOR.Decoder s (WithByteSpan a)
    decodeWithByteSpan :: forall s a. Decoder s a -> Decoder s (WithByteSpan a)
decodeWithByteSpan = ((a, Int64, Int64) -> WithByteSpan a)
-> Decoder s (a, Int64, Int64) -> Decoder s (WithByteSpan a)
forall a b. (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (a, Int64, Int64) -> WithByteSpan a
forall a. (a, Int64, Int64) -> WithByteSpan a
WithByteSpan (Decoder s (a, Int64, Int64) -> Decoder s (WithByteSpan a))
-> (Decoder s a -> Decoder s (a, Int64, Int64))
-> Decoder s a
-> Decoder s (WithByteSpan a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Decoder s a -> Decoder s (a, Int64, Int64)
forall s a. Decoder s a -> Decoder s (a, Int64, Int64)
CBOR.decodeWithByteSpan


--
-- Map protocol state & messages from `TxSubmission2 txid (Identity tx)` to
-- `TxSubmission2 txid tx`
--

type family FromIdentity (st :: TxSubmission2 txid (Identity tx)) :: TxSubmission2 txid tx where
  FromIdentity StInit        = StInit
  FromIdentity StIdle        = StIdle
  FromIdentity (StTxIds blk) = StTxIds blk
  FromIdentity StTxs         = StTxs
  FromIdentity StDone        = StDone

type family ToIdentity (st :: TxSubmission2 txid tx) :: TxSubmission2 txid (Identity tx) where
  ToIdentity StInit        = StInit
  ToIdentity StIdle        = StIdle
  ToIdentity (StTxIds blk) = StTxIds blk
  ToIdentity StTxs         = StTxs
  ToIdentity StDone        = StDone

singToIdentity
  :: forall txid tx (st :: TxSubmission2 txid tx).
     StateToken st
  -> StateToken (ToIdentity st)
singToIdentity :: forall txid tx (st :: TxSubmission2 txid tx).
StateToken st -> StateToken (ToIdentity st)
singToIdentity StateToken st
SingTxSubmission st
SingInit        = StateToken (ToIdentity st)
SingTxSubmission 'StInit
forall {txid} {tx}. SingTxSubmission 'StInit
SingInit
singToIdentity StateToken st
SingTxSubmission st
SingIdle        = StateToken (ToIdentity st)
SingTxSubmission 'StIdle
forall {txid} {tx}. SingTxSubmission 'StIdle
SingIdle
singToIdentity (SingTxIds SingBlockingStyle stBlocking
blk) = SingBlockingStyle stBlocking
-> SingTxSubmission ('StTxIds stBlocking)
forall {txid} {tx} (stBlocking :: StBlockingStyle).
SingBlockingStyle stBlocking
-> SingTxSubmission ('StTxIds stBlocking)
SingTxIds SingBlockingStyle stBlocking
blk
singToIdentity StateToken st
SingTxSubmission st
SingTxs         = StateToken (ToIdentity st)
SingTxSubmission 'StTxs
forall {txid} {tx}. SingTxSubmission 'StTxs
SingTxs
singToIdentity StateToken st
SingTxSubmission st
SingDone        = StateToken (ToIdentity st)
SingTxSubmission 'StDone
forall {txid} {tx}. SingTxSubmission 'StDone
SingDone

--
-- Proofs
--

-- | A proof that `FromIdentity` is a left inverse of `ToIdentity`.
--
proof_FromTo
  :: forall txid tx (st :: TxSubmission2 txid tx).
     StateToken st
  -> FromIdentity (ToIdentity st) :~: st
proof_FromTo :: forall txid tx (st :: TxSubmission2 txid tx).
StateToken st -> FromIdentity (ToIdentity st) :~: st
proof_FromTo StateToken st
SingTxSubmission st
SingInit    = st :~: st
FromIdentity (ToIdentity st) :~: st
forall {k} (a :: k). a :~: a
Refl
proof_FromTo StateToken st
SingTxSubmission st
SingIdle    = st :~: st
FromIdentity (ToIdentity st) :~: st
forall {k} (a :: k). a :~: a
Refl
proof_FromTo SingTxIds{} = st :~: st
FromIdentity (ToIdentity st) :~: st
forall {k} (a :: k). a :~: a
Refl
proof_FromTo StateToken st
SingTxSubmission st
SingTxs     = st :~: st
FromIdentity (ToIdentity st) :~: st
forall {k} (a :: k). a :~: a
Refl
proof_FromTo StateToken st
SingTxSubmission st
SingDone    = st :~: st
FromIdentity (ToIdentity st) :~: st
forall {k} (a :: k). a :~: a
Refl
{-# INLINE proof_FromTo #-}

-- | A proof that `ActiveState` constraint is preserved by `ToIdentity`.
--
proof_activeState
  :: forall txid tx (st :: TxSubmission2 txid tx).
     StateToken st
  -> Dict (ActiveState st)
  -> Dict (ActiveState (ToIdentity st))
proof_activeState :: forall txid tx (st :: TxSubmission2 txid tx).
StateToken st
-> Dict (ActiveState st) -> Dict (ActiveState (ToIdentity st))
proof_activeState StateToken st
SingTxSubmission st
SingInit      Dict (ActiveState st)
Dict = Dict (IsActiveState 'StInit 'ClientAgency)
Dict (IsActiveState (ToIdentity st) (StateAgency (ToIdentity st)))
forall (a :: Constraint). a => Dict a
Dict
proof_activeState StateToken st
SingTxSubmission st
SingIdle      Dict (ActiveState st)
Dict = Dict (IsActiveState 'StIdle 'ServerAgency)
Dict (IsActiveState (ToIdentity st) (StateAgency (ToIdentity st)))
forall (a :: Constraint). a => Dict a
Dict
proof_activeState SingTxIds{}   Dict (ActiveState st)
Dict = Dict (IsActiveState ('StTxIds stBlocking) 'ClientAgency)
Dict (IsActiveState (ToIdentity st) (StateAgency (ToIdentity st)))
forall (a :: Constraint). a => Dict a
Dict
proof_activeState StateToken st
SingTxSubmission st
SingTxs       Dict (ActiveState st)
Dict = Dict (IsActiveState 'StTxs 'ClientAgency)
Dict (IsActiveState (ToIdentity st) (StateAgency (ToIdentity st)))
forall (a :: Constraint). a => Dict a
Dict
proof_activeState sing :: StateToken st
sing@StateToken st
SingTxSubmission st
SingDone Dict (ActiveState st)
Dict = StateToken 'StDone -> forall a. a
forall ps (st :: ps).
(StateAgency st ~ 'NobodyAgency, ActiveState st) =>
StateToken st -> forall a. a
notActiveState StateToken st
StateToken 'StDone
sing
{-# INLINE proof_activeState #-}


msgFromIdentity
  :: forall txid tx (st :: TxSubmission2 txid (Identity tx)).
     SomeMessage st
  -> SomeMessage (FromIdentity st)
msgFromIdentity :: forall txid tx (st :: TxSubmission2 txid (Identity tx)).
SomeMessage st -> SomeMessage (FromIdentity st)
msgFromIdentity (SomeMessage Message (TxSubmission2 txid (Identity tx)) st st'
R:MessageTxSubmission2fromto txid (Identity tx) st st'
MsgInit)
              =  Message (TxSubmission2 txid tx) 'StInit 'StIdle
-> SomeMessage 'StInit
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) 'StInit 'StIdle
forall txid tx. Message (TxSubmission2 txid tx) 'StInit 'StIdle
MsgInit
msgFromIdentity (SomeMessage (MsgRequestTxIds blk :: SingBlockingStyle blocking
blk@SingBlockingStyle blocking
SingBlocking NumTxIdsToAck
ack NumTxIdsToReq
req))
              =  Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
-> SomeMessage 'StIdle
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
forall (blocking :: StBlockingStyle) txid tx.
SingI blocking =>
SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
MsgRequestTxIds SingBlockingStyle blocking
blk NumTxIdsToAck
ack NumTxIdsToReq
req)
msgFromIdentity (SomeMessage (MsgRequestTxIds blk :: SingBlockingStyle blocking
blk@SingBlockingStyle blocking
SingNonBlocking NumTxIdsToAck
ack NumTxIdsToReq
req))
              =  Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
-> SomeMessage 'StIdle
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
forall (blocking :: StBlockingStyle) txid tx.
SingI blocking =>
SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
MsgRequestTxIds SingBlockingStyle blocking
blk NumTxIdsToAck
ack NumTxIdsToReq
req)
msgFromIdentity (SomeMessage (MsgReplyTxIds txids :: BlockingReplyList blocking (txid, SizeInBytes)
txids@BlockingReply{}))
              =  Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
-> SomeMessage ('StTxIds blocking)
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
forall (blocking :: StBlockingStyle) txid tx.
BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
MsgReplyTxIds BlockingReplyList blocking (txid, SizeInBytes)
txids)
msgFromIdentity (SomeMessage (MsgReplyTxIds txids :: BlockingReplyList blocking (txid, SizeInBytes)
txids@NonBlockingReply{}))
              =  Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
-> SomeMessage ('StTxIds blocking)
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
forall (blocking :: StBlockingStyle) txid tx.
BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
MsgReplyTxIds BlockingReplyList blocking (txid, SizeInBytes)
txids)
msgFromIdentity (SomeMessage (MsgRequestTxs [txid]
txids))
              =  Message (TxSubmission2 txid tx) 'StIdle 'StTxs
-> SomeMessage 'StIdle
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage ([txid] -> Message (TxSubmission2 txid tx) 'StIdle 'StTxs
forall txid tx.
[txid] -> Message (TxSubmission2 txid tx) 'StIdle 'StTxs
MsgRequestTxs [txid]
txids)
msgFromIdentity (SomeMessage (MsgReplyTxs [Identity tx]
txs))
              =  Message (TxSubmission2 txid tx) 'StTxs 'StIdle
-> SomeMessage 'StTxs
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage ([tx] -> Message (TxSubmission2 txid tx) 'StTxs 'StIdle
forall tx txid.
[tx] -> Message (TxSubmission2 txid tx) 'StTxs 'StIdle
MsgReplyTxs (Identity tx -> tx
forall a. Identity a -> a
runIdentity (Identity tx -> tx) -> [Identity tx] -> [tx]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> [Identity tx]
txs))
msgFromIdentity (SomeMessage Message (TxSubmission2 txid (Identity tx)) st st'
R:MessageTxSubmission2fromto txid (Identity tx) st st'
MsgDone)
              =  Message (TxSubmission2 txid tx) ('StTxIds 'StBlocking) 'StDone
-> SomeMessage ('StTxIds 'StBlocking)
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) ('StTxIds 'StBlocking) 'StDone
forall txid tx.
Message (TxSubmission2 txid tx) ('StTxIds 'StBlocking) 'StDone
MsgDone
{-# INLINE msgFromIdentity #-}


codecTxSubmission2
  :: forall (txid :: Type) (tx :: Type) m.
     MonadST m
  => (txid -> CBOR.Encoding)
  -- ^ encode 'txid'
  -> (forall s . CBOR.Decoder s txid)
  -- ^ decode 'txid'
  -> (tx -> CBOR.Encoding)
  -- ^ encode transaction
  -> (forall s . CBOR.Decoder s tx)
  -- ^ decode transaction
  -> Codec (TxSubmission2 txid tx) CBOR.DeserialiseFailure m ByteString
codecTxSubmission2 :: forall txid tx (m :: * -> *).
MonadST m =>
(txid -> Encoding)
-> (forall s. Decoder s txid)
-> (tx -> Encoding)
-> (forall s. Decoder s tx)
-> Codec (TxSubmission2 txid tx) DeserialiseFailure m ByteString
codecTxSubmission2 txid -> Encoding
encodeTxId forall s. Decoder s txid
decodeTxId
                   tx -> Encoding
encodeTx   forall s. Decoder s tx
decodeTx
  = (forall (st :: TxSubmission2 txid tx)
        (st' :: TxSubmission2 txid tx).
 (StateTokenI st, ActiveState st) =>
 Message (TxSubmission2 txid tx) st st' -> Encoding)
-> (forall (st :: TxSubmission2 txid tx) s.
    ActiveState st =>
    StateToken st -> Decoder s (SomeMessage st))
-> CodecF
     (TxSubmission2 txid tx) DeserialiseFailure m SomeMessage ByteString
forall ps (m :: * -> *) (f :: ps -> *).
MonadST m =>
(forall (st :: ps) (st' :: ps).
 (StateTokenI st, ActiveState st) =>
 Message ps st st' -> Encoding)
-> (forall (st :: ps) s.
    ActiveState st =>
    StateToken st -> Decoder s (f st))
-> CodecF ps DeserialiseFailure m f ByteString
mkCodecCborLazyBS
      ((txid -> Encoding)
-> (tx -> Encoding)
-> Message (TxSubmission2 txid tx) st st'
-> Encoding
forall txid tx (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(txid -> Encoding)
-> (tx -> Encoding)
-> Message (TxSubmission2 txid tx) st st'
-> Encoding
encodeTxSubmission2 txid -> Encoding
encodeTxId tx -> Encoding
encodeTx)
      StateToken st -> Decoder s (SomeMessage st)
forall (st :: TxSubmission2 txid tx) s.
ActiveState st =>
StateToken st -> Decoder s (SomeMessage st)
decode
  where
    decode :: forall (st :: TxSubmission2 txid tx) s.
                         ActiveState st
                      => StateToken st
                      -> CBOR.Decoder s (SomeMessage st)
    decode :: forall (st :: TxSubmission2 txid tx) s.
ActiveState st =>
StateToken st -> Decoder s (SomeMessage st)
decode StateToken st
sing =
      case StateToken st -> FromIdentity (ToIdentity st) :~: st
forall txid tx (st :: TxSubmission2 txid tx).
StateToken st -> FromIdentity (ToIdentity st) :~: st
proof_FromTo StateToken st
sing of { FromIdentity (ToIdentity st) :~: st
Refl ->
        case StateToken st
-> Dict (ActiveState st) -> Dict (ActiveState (ToIdentity st))
forall txid tx (st :: TxSubmission2 txid tx).
StateToken st
-> Dict (ActiveState st) -> Dict (ActiveState (ToIdentity st))
proof_activeState StateToken st
sing Dict (ActiveState st)
forall (a :: Constraint). a => Dict a
Dict of { Dict (ActiveState (ToIdentity st))
Dict ->
          -- The `decode'` function requires the `ActiveState (Identity st)` constraint,
          -- which is provided by the `proof_activeState` proof for the given
          -- state token.
          --
          -- The `msgFromIdentity` function is applied to transform the result of `decode'`
          -- into a value of type `SomeMessage (FromIdentity (ToIdentity st))`. The
          -- `proof_FromTo` proof is then used to establish the type equality between
          -- `SomeMessage (FromIdentity (ToIdentity st))` and `SomeMessage st`, allowing
          -- GHC to infer the correct type.
          SomeMessage (ToIdentity st) -> SomeMessage st
SomeMessage (ToIdentity st)
-> SomeMessage (FromIdentity (ToIdentity st))
forall txid tx (st :: TxSubmission2 txid (Identity tx)).
SomeMessage st -> SomeMessage (FromIdentity st)
msgFromIdentity (SomeMessage (ToIdentity st) -> SomeMessage st)
-> Decoder s (SomeMessage (ToIdentity st))
-> Decoder s (SomeMessage st)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StateToken (ToIdentity st)
-> forall s. Decoder s (SomeMessage (ToIdentity st))
forall (st :: TxSubmission2 txid (Identity tx)).
ActiveState st =>
StateToken st -> forall s. Decoder s (SomeMessage st)
decode' (StateToken st -> StateToken (ToIdentity st)
forall txid tx (st :: TxSubmission2 txid tx).
StateToken st -> StateToken (ToIdentity st)
singToIdentity StateToken st
sing)
        }
      }

    decode' :: forall (st :: TxSubmission2 txid (Identity tx)).
              ActiveState st
           => StateToken st
           -> forall s. CBOR.Decoder s (SomeMessage st)
    decode' :: forall (st :: TxSubmission2 txid (Identity tx)).
ActiveState st =>
StateToken st -> forall s. Decoder s (SomeMessage st)
decode' StateToken st
stok =
      Annotator () st -> SomeMessage st
forall {ps} (st :: ps). Annotator () st -> SomeMessage st
mapAnnotator (Annotator () st -> SomeMessage st)
-> Decoder s (Annotator () st) -> Decoder s (SomeMessage st)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
        forall (withBytes :: * -> *) (withByteSpan :: * -> *) bytes txid tx
       (st :: TxSubmission2 txid (withBytes tx)) s.
ActiveState st =>
(bytes -> withByteSpan tx -> withBytes tx)
-> (forall a s'. Decoder s' a -> Decoder s' (withByteSpan a))
-> (forall s'. Decoder s' txid)
-> (forall s'. Decoder s' tx)
-> StateToken st
-> Decoder s (Annotator bytes st)
decodeTxSubmission2 @Identity
                            @Identity
                            @()
                            () -> Identity tx -> Identity tx
mkWithBytes
                            Decoder s' a -> Decoder s' (Identity a)
forall s a. Decoder s a -> Decoder s (Identity a)
forall a s'. Decoder s' a -> Decoder s' (Identity a)
decodeWithByteSpan
                            Decoder s' txid
forall s. Decoder s txid
decodeTxId Decoder s' tx
forall s. Decoder s tx
decodeTx
                            StateToken st
stok

    mapAnnotator :: Annotator () st -> SomeMessage st
    mapAnnotator :: forall {ps} (st :: ps). Annotator () st -> SomeMessage st
mapAnnotator Annotator { () -> SomeMessage st
runAnnotator :: () -> SomeMessage st
runAnnotator :: forall {ps} bytes (st :: ps).
Annotator bytes st -> bytes -> SomeMessage st
runAnnotator } = () -> SomeMessage st
runAnnotator ()

    mkWithBytes :: () -> Identity tx -> Identity tx
    mkWithBytes :: () -> Identity tx -> Identity tx
mkWithBytes ()
_ = Identity tx -> Identity tx
forall a. a -> a
id

    decodeWithByteSpan :: CBOR.Decoder s a -> CBOR.Decoder s (Identity a)
    decodeWithByteSpan :: forall s a. Decoder s a -> Decoder s (Identity a)
decodeWithByteSpan = (a -> Identity a) -> Decoder s a -> Decoder s (Identity a)
forall a b. (a -> b) -> Decoder s a -> Decoder s b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap a -> Identity a
forall a. a -> Identity a
Identity


encodeTxSubmission2
    :: forall (txid :: Type) (tx :: Type) (st :: TxSubmission2 txid tx) (st' :: TxSubmission2 txid tx).
       (txid -> CBOR.Encoding)
    -- ^ encode 'txid'
    -> (tx -> CBOR.Encoding)
    -- ^ encode 'tx'
    -> Message (TxSubmission2 txid tx) st st'
    -> CBOR.Encoding
encodeTxSubmission2 :: forall txid tx (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(txid -> Encoding)
-> (tx -> Encoding)
-> Message (TxSubmission2 txid tx) st st'
-> Encoding
encodeTxSubmission2 txid -> Encoding
encodeTxId tx -> Encoding
encodeTx = Message (TxSubmission2 txid tx) st st' -> Encoding
forall (st0 :: TxSubmission2 txid tx)
       (st1 :: TxSubmission2 txid tx).
Message (TxSubmission2 txid tx) st0 st1 -> Encoding
encode
  where
    encode :: forall st0 st1.
              Message (TxSubmission2 txid tx) st0 st1
           -> CBOR.Encoding
    encode :: forall (st0 :: TxSubmission2 txid tx)
       (st1 :: TxSubmission2 txid tx).
Message (TxSubmission2 txid tx) st0 st1 -> Encoding
encode Message (TxSubmission2 txid tx) st0 st1
R:MessageTxSubmission2fromto txid tx st0 st1
MsgInit =
        Word -> Encoding
CBOR.encodeListLen Word
1
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
6
    encode (MsgRequestTxIds SingBlockingStyle blocking
blocking (NumTxIdsToAck Word16
ackNo) (NumTxIdsToReq Word16
reqNo)) =
        Word -> Encoding
CBOR.encodeListLen Word
4
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
0
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Bool -> Encoding
CBOR.encodeBool (case SingBlockingStyle blocking
blocking of
                           SingBlockingStyle blocking
SingBlocking    -> Bool
True
                           SingBlockingStyle blocking
SingNonBlocking -> Bool
False)
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word16 -> Encoding
CBOR.encodeWord16 Word16
ackNo
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word16 -> Encoding
CBOR.encodeWord16 Word16
reqNo

    encode (MsgReplyTxIds BlockingReplyList blocking (txid, SizeInBytes)
txids) =
        Word -> Encoding
CBOR.encodeListLen Word
2
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
1
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
CBOR.encodeListLenIndef
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> ((txid, SizeInBytes) -> Encoding -> Encoding)
-> Encoding -> [(txid, SizeInBytes)] -> Encoding
forall a b. (a -> b -> b) -> b -> [a] -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\(txid
txid, SizeInBytes Word32
sz) Encoding
r ->
                                Word -> Encoding
CBOR.encodeListLen Word
2
                             Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> txid -> Encoding
encodeTxId txid
txid
                             Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word32 -> Encoding
CBOR.encodeWord32 Word32
sz
                             Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
r)
              Encoding
CBOR.encodeBreak
              [(txid, SizeInBytes)]
txids'
      where
        txids' :: [(txid, SizeInBytes)]
        txids' :: [(txid, SizeInBytes)]
txids' = case BlockingReplyList blocking (txid, SizeInBytes)
txids of
                   BlockingReply    NonEmpty (txid, SizeInBytes)
xs -> NonEmpty (txid, SizeInBytes) -> [(txid, SizeInBytes)]
forall a. NonEmpty a -> [a]
NonEmpty.toList NonEmpty (txid, SizeInBytes)
xs
                   NonBlockingReply [(txid, SizeInBytes)]
xs -> [(txid, SizeInBytes)]
xs

    encode (MsgRequestTxs [txid]
txids) =
        Word -> Encoding
CBOR.encodeListLen Word
2
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
2
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
CBOR.encodeListLenIndef
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> (txid -> Encoding -> Encoding) -> Encoding -> [txid] -> Encoding
forall a b. (a -> b -> b) -> b -> [a] -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\txid
txid Encoding
r -> txid -> Encoding
encodeTxId txid
txid Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
r) Encoding
CBOR.encodeBreak [txid]
txids

    encode (MsgReplyTxs [tx]
txs) =
        Word -> Encoding
CBOR.encodeListLen Word
2
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
3
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
CBOR.encodeListLenIndef
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> (tx -> Encoding -> Encoding) -> Encoding -> [tx] -> Encoding
forall a b. (a -> b -> b) -> b -> [a] -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
foldr (\tx
txid Encoding
r -> tx -> Encoding
encodeTx tx
txid Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Encoding
r) Encoding
CBOR.encodeBreak [tx]
txs

    encode Message (TxSubmission2 txid tx) st0 st1
R:MessageTxSubmission2fromto txid tx st0 st1
MsgDone =
        Word -> Encoding
CBOR.encodeListLen Word
1
     Encoding -> Encoding -> Encoding
forall a. Semigroup a => a -> a -> a
<> Word -> Encoding
CBOR.encodeWord Word
4


-- | Decode tx-submission mini-protocol message.  This decoder is polymorphic
-- in `txid`, `tx`, but also in:
--
-- * annotator's bytes
-- * tx's annotation, e.g. `withBytes`
-- * offsets used by the tx's annotator, e.g. `byteOffsetF`
--
decodeTxSubmission2
    :: forall (withBytes    :: Type -> Type) -- tx annotator functor
              (withByteSpan :: Type -> Type) -- withByteSpan functor
              (bytes        :: Type)         -- annotation bytes
              (txid :: Type) (tx :: Type)
              (st :: TxSubmission2 txid (withBytes tx))
              s.
       ActiveState st
    => (bytes -> withByteSpan tx -> withBytes tx)
    -- ^ mkWithBytes: smart constructor for `withBytes` functor.
    -> (forall a s'. CBOR.Decoder s' a -> CBOR.Decoder s' (withByteSpan a))
    -- ^ turn a `CBOR.Decoder` into a decoder of `withByteSpan a`, e.g.
    -- `CBOR.decodeWithByteSpan`.
    -> (forall s'. CBOR.Decoder s' txid)
    -- ^ decode a transaction id
    -> (forall s'. CBOR.Decoder s' tx)
    -- ^ decode a transaction
    -> StateToken st
    -- ^ protocol state token
    -> CBOR.Decoder s (Annotator bytes st)
decodeTxSubmission2 :: forall (withBytes :: * -> *) (withByteSpan :: * -> *) bytes txid tx
       (st :: TxSubmission2 txid (withBytes tx)) s.
ActiveState st =>
(bytes -> withByteSpan tx -> withBytes tx)
-> (forall a s'. Decoder s' a -> Decoder s' (withByteSpan a))
-> (forall s'. Decoder s' txid)
-> (forall s'. Decoder s' tx)
-> StateToken st
-> Decoder s (Annotator bytes st)
decodeTxSubmission2 bytes -> withByteSpan tx -> withBytes tx
mkWithBytes forall a s'. Decoder s' a -> Decoder s' (withByteSpan a)
decodeWithByteSpan
                    forall s'. Decoder s' txid
decodeTxId forall s'. Decoder s' tx
decodeTx
                    StateToken st
stok
  = do
  len <- Decoder s Int
forall s. Decoder s Int
CBOR.decodeListLen
  key <- CBOR.decodeWord
  case (stok, len, key) of
    (SingTxSubmission st
SingInit, Int
1, Word
6) ->
      Annotator bytes st -> Decoder s (Annotator bytes st)
forall a. a -> Decoder s a
forall (m :: * -> *) a. Monad m => a -> m a
return ((bytes -> SomeMessage st) -> Annotator bytes st
forall {ps} bytes (st :: ps).
(bytes -> SomeMessage st) -> Annotator bytes st
Annotator ((bytes -> SomeMessage st) -> Annotator bytes st)
-> (bytes -> SomeMessage st) -> Annotator bytes st
forall a b. (a -> b) -> a -> b
$ \bytes
_ -> Message (TxSubmission2 txid (withBytes tx)) st 'StIdle
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid (withBytes tx)) st 'StIdle
Message (TxSubmission2 txid (withBytes tx)) 'StInit 'StIdle
forall txid tx. Message (TxSubmission2 txid tx) 'StInit 'StIdle
MsgInit)

    (SingTxSubmission st
SingIdle, Int
4, Word
0) -> do
      blocking <- Decoder s Bool
forall s. Decoder s Bool
CBOR.decodeBool
      ackNo    <- NumTxIdsToAck <$> CBOR.decodeWord16
      reqNo    <- NumTxIdsToReq <$> CBOR.decodeWord16
      return $!
        if blocking
        then Annotator $ \bytes
_ -> Message
  (TxSubmission2 txid (withBytes tx)) st ('StTxIds 'StBlocking)
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (SingBlockingStyle 'StBlocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message
     (TxSubmission2 txid (withBytes tx)) 'StIdle ('StTxIds 'StBlocking)
forall (blocking :: StBlockingStyle) txid tx.
SingI blocking =>
SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
MsgRequestTxIds SingBlockingStyle 'StBlocking
SingBlocking    NumTxIdsToAck
ackNo NumTxIdsToReq
reqNo)
        else Annotator $ \bytes
_ -> Message
  (TxSubmission2 txid (withBytes tx)) st ('StTxIds 'StNonBlocking)
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (SingBlockingStyle 'StNonBlocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message
     (TxSubmission2 txid (withBytes tx))
     'StIdle
     ('StTxIds 'StNonBlocking)
forall (blocking :: StBlockingStyle) txid tx.
SingI blocking =>
SingBlockingStyle blocking
-> NumTxIdsToAck
-> NumTxIdsToReq
-> Message (TxSubmission2 txid tx) 'StIdle ('StTxIds blocking)
MsgRequestTxIds SingBlockingStyle 'StNonBlocking
SingNonBlocking NumTxIdsToAck
ackNo NumTxIdsToReq
reqNo)

    (SingTxIds SingBlockingStyle stBlocking
b, Int
2, Word
1) -> do
      Decoder s ()
forall s. Decoder s ()
CBOR.decodeListLenIndef
      txids <- ([(txid, SizeInBytes)]
 -> (txid, SizeInBytes) -> [(txid, SizeInBytes)])
-> [(txid, SizeInBytes)]
-> ([(txid, SizeInBytes)] -> [(txid, SizeInBytes)])
-> Decoder s (txid, SizeInBytes)
-> Decoder s [(txid, SizeInBytes)]
forall r a r' s.
(r -> a -> r) -> r -> (r -> r') -> Decoder s a -> Decoder s r'
CBOR.decodeSequenceLenIndef
                 (((txid, SizeInBytes)
 -> [(txid, SizeInBytes)] -> [(txid, SizeInBytes)])
-> [(txid, SizeInBytes)]
-> (txid, SizeInBytes)
-> [(txid, SizeInBytes)]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (:)) [] [(txid, SizeInBytes)] -> [(txid, SizeInBytes)]
forall a. [a] -> [a]
reverse
                 (do Int -> Decoder s ()
forall s. Int -> Decoder s ()
CBOR.decodeListLenOf Int
2
                     txid <- Decoder s txid
forall s'. Decoder s' txid
decodeTxId
                     sz   <- CBOR.decodeWord32
                     return (txid, SizeInBytes sz))
      case (b, txids) of
        (SingBlockingStyle stBlocking
SingBlocking, (txid, SizeInBytes)
t:[(txid, SizeInBytes)]
ts) ->
          Annotator bytes st -> Decoder s (Annotator bytes st)
forall a. a -> Decoder s a
forall (m :: * -> *) a. Monad m => a -> m a
return (Annotator bytes st -> Decoder s (Annotator bytes st))
-> Annotator bytes st -> Decoder s (Annotator bytes st)
forall a b. (a -> b) -> a -> b
$ (bytes -> SomeMessage st) -> Annotator bytes st
forall {ps} bytes (st :: ps).
(bytes -> SomeMessage st) -> Annotator bytes st
Annotator ((bytes -> SomeMessage st) -> Annotator bytes st)
-> (bytes -> SomeMessage st) -> Annotator bytes st
forall a b. (a -> b) -> a -> b
$ \bytes
_ ->
            Message (TxSubmission2 txid (withBytes tx)) st 'StIdle
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (BlockingReplyList 'StBlocking (txid, SizeInBytes)
-> Message
     (TxSubmission2 txid (withBytes tx)) ('StTxIds 'StBlocking) 'StIdle
forall (blocking :: StBlockingStyle) txid tx.
BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
MsgReplyTxIds (NonEmpty (txid, SizeInBytes)
-> BlockingReplyList 'StBlocking (txid, SizeInBytes)
forall a. NonEmpty a -> BlockingReplyList 'StBlocking a
BlockingReply ((txid, SizeInBytes)
t (txid, SizeInBytes)
-> [(txid, SizeInBytes)] -> NonEmpty (txid, SizeInBytes)
forall a. a -> [a] -> NonEmpty a
NonEmpty.:| [(txid, SizeInBytes)]
ts)))

        (SingBlockingStyle stBlocking
SingNonBlocking, [(txid, SizeInBytes)]
ts) ->
          Annotator bytes st -> Decoder s (Annotator bytes st)
forall a. a -> Decoder s a
forall (m :: * -> *) a. Monad m => a -> m a
return (Annotator bytes st -> Decoder s (Annotator bytes st))
-> Annotator bytes st -> Decoder s (Annotator bytes st)
forall a b. (a -> b) -> a -> b
$ (bytes -> SomeMessage st) -> Annotator bytes st
forall {ps} bytes (st :: ps).
(bytes -> SomeMessage st) -> Annotator bytes st
Annotator ((bytes -> SomeMessage st) -> Annotator bytes st)
-> (bytes -> SomeMessage st) -> Annotator bytes st
forall a b. (a -> b) -> a -> b
$ \bytes
_ ->
            Message (TxSubmission2 txid (withBytes tx)) st 'StIdle
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage (BlockingReplyList 'StNonBlocking (txid, SizeInBytes)
-> Message
     (TxSubmission2 txid (withBytes tx))
     ('StTxIds 'StNonBlocking)
     'StIdle
forall (blocking :: StBlockingStyle) txid tx.
BlockingReplyList blocking (txid, SizeInBytes)
-> Message (TxSubmission2 txid tx) ('StTxIds blocking) 'StIdle
MsgReplyTxIds ([(txid, SizeInBytes)]
-> BlockingReplyList 'StNonBlocking (txid, SizeInBytes)
forall a. [a] -> BlockingReplyList 'StNonBlocking a
NonBlockingReply [(txid, SizeInBytes)]
ts))

        (SingBlockingStyle stBlocking
SingBlocking, []) ->
          String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail String
"codecTxSubmission: MsgReplyTxIds: empty list not permitted"

    (SingTxSubmission st
SingIdle, Int
2, Word
2) -> do
      Decoder s ()
forall s. Decoder s ()
CBOR.decodeListLenIndef
      txids <- ([txid] -> txid -> [txid])
-> [txid]
-> ([txid] -> [txid])
-> Decoder s txid
-> Decoder s [txid]
forall r a r' s.
(r -> a -> r) -> r -> (r -> r') -> Decoder s a -> Decoder s r'
CBOR.decodeSequenceLenIndef ((txid -> [txid] -> [txid]) -> [txid] -> txid -> [txid]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (:)) [] [txid] -> [txid]
forall a. [a] -> [a]
reverse Decoder s txid
forall s'. Decoder s' txid
decodeTxId
      return (Annotator $ \bytes
_ -> Message (TxSubmission2 txid (withBytes tx)) st 'StTxs
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage ([txid]
-> Message (TxSubmission2 txid (withBytes tx)) 'StIdle 'StTxs
forall txid tx.
[txid] -> Message (TxSubmission2 txid tx) 'StIdle 'StTxs
MsgRequestTxs [txid]
txids))

    (SingTxSubmission st
SingTxs, Int
2, Word
3) -> do
      Decoder s ()
forall s. Decoder s ()
CBOR.decodeListLenIndef
      txs <- ([withByteSpan tx] -> withByteSpan tx -> [withByteSpan tx])
-> [withByteSpan tx]
-> ([withByteSpan tx] -> [withByteSpan tx])
-> Decoder s (withByteSpan tx)
-> Decoder s [withByteSpan tx]
forall r a r' s.
(r -> a -> r) -> r -> (r -> r') -> Decoder s a -> Decoder s r'
CBOR.decodeSequenceLenIndef ((withByteSpan tx -> [withByteSpan tx] -> [withByteSpan tx])
-> [withByteSpan tx] -> withByteSpan tx -> [withByteSpan tx]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (:)) [] [withByteSpan tx] -> [withByteSpan tx]
forall a. [a] -> [a]
reverse (Decoder s tx -> Decoder s (withByteSpan tx)
forall a s'. Decoder s' a -> Decoder s' (withByteSpan a)
decodeWithByteSpan Decoder s tx
forall s'. Decoder s' tx
decodeTx)
      return (Annotator  $
                \bytes
bytes -> Message (TxSubmission2 txid (withBytes tx)) st 'StIdle
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage ([withBytes tx]
-> Message (TxSubmission2 txid (withBytes tx)) 'StTxs 'StIdle
forall tx txid.
[tx] -> Message (TxSubmission2 txid tx) 'StTxs 'StIdle
MsgReplyTxs ([withBytes tx]
 -> Message (TxSubmission2 txid (withBytes tx)) 'StTxs 'StIdle)
-> [withBytes tx]
-> Message (TxSubmission2 txid (withBytes tx)) 'StTxs 'StIdle
forall a b. (a -> b) -> a -> b
$ bytes -> withByteSpan tx -> withBytes tx
mkWithBytes bytes
bytes (withByteSpan tx -> withBytes tx)
-> [withByteSpan tx] -> [withBytes tx]
forall a b. (a -> b) -> [a] -> [b]
`map` [withByteSpan tx]
txs))

    (SingTxIds SingBlockingStyle stBlocking
SingBlocking, Int
1, Word
4) ->
      Annotator bytes st -> Decoder s (Annotator bytes st)
forall a. a -> Decoder s a
forall (m :: * -> *) a. Monad m => a -> m a
return ((bytes -> SomeMessage st) -> Annotator bytes st
forall {ps} bytes (st :: ps).
(bytes -> SomeMessage st) -> Annotator bytes st
Annotator ((bytes -> SomeMessage st) -> Annotator bytes st)
-> (bytes -> SomeMessage st) -> Annotator bytes st
forall a b. (a -> b) -> a -> b
$ \bytes
_ -> Message (TxSubmission2 txid (withBytes tx)) st 'StDone
-> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid (withBytes tx)) st 'StDone
Message
  (TxSubmission2 txid (withBytes tx)) ('StTxIds 'StBlocking) 'StDone
forall txid tx.
Message (TxSubmission2 txid tx) ('StTxIds 'StBlocking) 'StDone
MsgDone)

    (SingTxSubmission st
SingDone, Int
_, Word
_) -> StateToken 'StDone -> forall a. a
forall ps (st :: ps).
(StateAgency st ~ 'NobodyAgency, ActiveState st) =>
StateToken st -> forall a. a
notActiveState StateToken st
StateToken 'StDone
stok

    --
    -- failures per protocol state
    --

    (SingTxSubmission st
SingInit, Int
_, Word
_) ->
        String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (String -> String -> Word -> Int -> String
forall r. PrintfType r => String -> r
printf String
"codecTxSubmission (%s) unexpected key (%d, %d)" (SingTxSubmission 'StInit -> String
forall a. Show a => a -> String
show StateToken st
SingTxSubmission 'StInit
stok) Word
key Int
len)
    (SingTxIds SingBlockingStyle stBlocking
SingBlocking, Int
_, Word
_) ->
        String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (String -> String -> Word -> Int -> String
forall r. PrintfType r => String -> r
printf String
"codecTxSubmission (%s) unexpected key (%d, %d)" (SingTxSubmission ('StTxIds 'StBlocking) -> String
forall a. Show a => a -> String
show StateToken st
SingTxSubmission ('StTxIds 'StBlocking)
stok) Word
key Int
len)
    (SingTxIds SingBlockingStyle stBlocking
SingNonBlocking, Int
_, Word
_) ->
        String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (String -> String -> Word -> Int -> String
forall r. PrintfType r => String -> r
printf String
"codecTxSubmission (%s) unexpected key (%d, %d)" (SingTxSubmission ('StTxIds 'StNonBlocking) -> String
forall a. Show a => a -> String
show StateToken st
SingTxSubmission ('StTxIds 'StNonBlocking)
stok) Word
key Int
len)
    (SingTxSubmission st
SingTxs, Int
_, Word
_) ->
        String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (String -> String -> Word -> Int -> String
forall r. PrintfType r => String -> r
printf String
"codecTxSubmission (%s) unexpected key (%d, %d)" (SingTxSubmission 'StTxs -> String
forall a. Show a => a -> String
show StateToken st
SingTxSubmission 'StTxs
stok) Word
key Int
len)
    (SingTxSubmission st
SingIdle, Int
_, Word
_) ->
        String -> Decoder s (Annotator bytes st)
forall a. String -> Decoder s a
forall (m :: * -> *) a. MonadFail m => String -> m a
fail (String -> String -> Word -> Int -> String
forall r. PrintfType r => String -> r
printf String
"codecTxSubmission (%s) unexpected key (%d, %d)" (SingTxSubmission 'StIdle -> String
forall a. Show a => a -> String
show StateToken st
SingTxSubmission 'StIdle
stok) Word
key Int
len)


codecTxSubmission2Id
  :: forall txid tx m. Monad m
  => Codec (TxSubmission2 txid tx) CodecFailure m (AnyMessage (TxSubmission2 txid tx))
codecTxSubmission2Id :: forall txid tx (m :: * -> *).
Monad m =>
Codec
  (TxSubmission2 txid tx)
  CodecFailure
  m
  (AnyMessage (TxSubmission2 txid tx))
codecTxSubmission2Id = Codec { Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
forall (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(ActiveState st, StateTokenI st) =>
Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
forall (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(StateTokenI st, ActiveState st) =>
Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
encode :: forall (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(ActiveState st, StateTokenI st) =>
Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
encode :: forall (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(StateTokenI st, ActiveState st) =>
Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
encode, StateToken st
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
decode :: forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
decode :: forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
decode }
 where
  encode :: forall st st'.
            ActiveState st
         => StateTokenI st
         => Message (TxSubmission2 txid tx) st st'
         -> AnyMessage (TxSubmission2 txid tx)
  encode :: forall (st :: TxSubmission2 txid tx)
       (st' :: TxSubmission2 txid tx).
(ActiveState st, StateTokenI st) =>
Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
encode = Message (TxSubmission2 txid tx) st st'
-> AnyMessage (TxSubmission2 txid tx)
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, ActiveState st) =>
Message ps st st' -> AnyMessage ps
AnyMessage

  decode :: forall (st :: TxSubmission2 txid tx).
            ActiveState st
         => StateToken st
         -> m (DecodeStep (AnyMessage (TxSubmission2 txid tx))
                          CodecFailure m (SomeMessage st))
  decode :: forall (st :: TxSubmission2 txid tx).
ActiveState st =>
StateToken st
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
decode StateToken st
stok = DecodeStep
  (AnyMessage (TxSubmission2 txid tx))
  CodecFailure
  m
  (SomeMessage st)
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (DecodeStep
   (AnyMessage (TxSubmission2 txid tx))
   CodecFailure
   m
   (SomeMessage st)
 -> m (DecodeStep
         (AnyMessage (TxSubmission2 txid tx))
         CodecFailure
         m
         (SomeMessage st)))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
forall a b. (a -> b) -> a -> b
$ (Maybe (AnyMessage (TxSubmission2 txid tx))
 -> m (DecodeStep
         (AnyMessage (TxSubmission2 txid tx))
         CodecFailure
         m
         (SomeMessage st)))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
(Maybe bytes -> m (DecodeStep bytes failure m a))
-> DecodeStep bytes failure m a
DecodePartial ((Maybe (AnyMessage (TxSubmission2 txid tx))
  -> m (DecodeStep
          (AnyMessage (TxSubmission2 txid tx))
          CodecFailure
          m
          (SomeMessage st)))
 -> DecodeStep
      (AnyMessage (TxSubmission2 txid tx))
      CodecFailure
      m
      (SomeMessage st))
-> (Maybe (AnyMessage (TxSubmission2 txid tx))
    -> m (DecodeStep
            (AnyMessage (TxSubmission2 txid tx))
            CodecFailure
            m
            (SomeMessage st)))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall a b. (a -> b) -> a -> b
$ \Maybe (AnyMessage (TxSubmission2 txid tx))
bytes -> DecodeStep
  (AnyMessage (TxSubmission2 txid tx))
  CodecFailure
  m
  (SomeMessage st)
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (DecodeStep
   (AnyMessage (TxSubmission2 txid tx))
   CodecFailure
   m
   (SomeMessage st)
 -> m (DecodeStep
         (AnyMessage (TxSubmission2 txid tx))
         CodecFailure
         m
         (SomeMessage st)))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
-> m (DecodeStep
        (AnyMessage (TxSubmission2 txid tx))
        CodecFailure
        m
        (SomeMessage st))
forall a b. (a -> b) -> a -> b
$ case (StateToken st
SingTxSubmission st
stok, Maybe (AnyMessage (TxSubmission2 txid tx))
bytes) of
    (SingTxSubmission st
SingInit,      Just (AnyMessage msg :: Message (TxSubmission2 txid tx) st st'
msg@Message (TxSubmission2 txid tx) st st'
R:MessageTxSubmission2fromto txid tx st st'
MsgInit))              -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
    (SingTxSubmission st
SingIdle,      Just (AnyMessage msg :: Message (TxSubmission2 txid tx) st st'
msg@(MsgRequestTxIds SingBlockingStyle blocking
SingBlocking NumTxIdsToAck
_ NumTxIdsToReq
_)))    -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
    (SingTxSubmission st
SingIdle,      Just (AnyMessage msg :: Message (TxSubmission2 txid tx) st st'
msg@(MsgRequestTxIds SingBlockingStyle blocking
SingNonBlocking NumTxIdsToAck
_ NumTxIdsToReq
_))) -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
    (SingTxSubmission st
SingIdle,      Just (AnyMessage msg :: Message (TxSubmission2 txid tx) st st'
msg@(MsgRequestTxs {})))   -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
    (SingTxSubmission st
SingTxs,       Just (AnyMessage msg :: Message (TxSubmission2 txid tx) st st'
msg@(MsgReplyTxs {})))     -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
    (SingTxIds SingBlockingStyle stBlocking
b, Just (AnyMessage Message (TxSubmission2 txid tx) st st'
msg)) -> case (SingBlockingStyle stBlocking
b, Message (TxSubmission2 txid tx) st st'
msg) of
      (SingBlockingStyle stBlocking
SingBlocking,    MsgReplyTxIds (BlockingReply {}))    -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
      (SingBlockingStyle stBlocking
SingNonBlocking, MsgReplyTxIds (NonBlockingReply {})) -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
      (SingBlockingStyle stBlocking
SingBlocking,    MsgDone {})                          -> SomeMessage st
-> Maybe (AnyMessage (TxSubmission2 txid tx))
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
a -> Maybe bytes -> DecodeStep bytes failure m a
DecodeDone (Message (TxSubmission2 txid tx) st st' -> SomeMessage st
forall ps (st :: ps) (st' :: ps).
(StateTokenI st, StateTokenI st', ActiveState st) =>
Message ps st st' -> SomeMessage st
SomeMessage Message (TxSubmission2 txid tx) st st'
Message (TxSubmission2 txid tx) st st'
msg) Maybe (AnyMessage (TxSubmission2 txid tx))
forall a. Maybe a
Nothing
      (SingBlockingStyle stBlocking
_, Message (TxSubmission2 txid tx) st st'
_) -> CodecFailure
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
failure -> DecodeStep bytes failure m a
DecodeFail (String -> CodecFailure
CodecFailure String
"codecTxSubmissionId: no matching message")
    (SingTxSubmission st
SingDone, Maybe (AnyMessage (TxSubmission2 txid tx))
_) -> StateToken 'StDone -> forall a. a
forall ps (st :: ps).
(StateAgency st ~ 'NobodyAgency, ActiveState st) =>
StateToken st -> forall a. a
notActiveState StateToken st
StateToken 'StDone
stok
    (SingTxSubmission st
_, Maybe (AnyMessage (TxSubmission2 txid tx))
_) -> CodecFailure
-> DecodeStep
     (AnyMessage (TxSubmission2 txid tx))
     CodecFailure
     m
     (SomeMessage st)
forall bytes failure (m :: * -> *) a.
failure -> DecodeStep bytes failure m a
DecodeFail (String -> CodecFailure
CodecFailure String
"codecTxSubmissionId: no matching message")