Skip to content

Commit

Permalink
GRPC BlockItems (#540)
Browse files Browse the repository at this point in the history
Add a GetBlockItems method to retrieve the list of transactions in a block as-is (as opposed to outcomes).

Co-authored-by: Aleš Bizjak <[email protected]>
  • Loading branch information
MilkywayPirate and abizjak authored Oct 10, 2022
1 parent 586dc2b commit 6be4de4
Show file tree
Hide file tree
Showing 7 changed files with 224 additions and 10 deletions.
135 changes: 131 additions & 4 deletions concordium-consensus/src/Concordium/GRPC2.hs
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import Concordium.Types
import Concordium.Types.Accounts
import qualified Concordium.Types.Transactions as TxTypes
import qualified Concordium.Types.Queries as QueryTypes
import qualified Concordium.Types.Transactions as Transactions
import qualified Concordium.GlobalState.ContractStateV1 as StateV1
import qualified Concordium.Queries as Q

Expand All @@ -54,7 +55,7 @@ import Concordium.ID.Parameters (withGlobalContext)
import Concordium.Common.Time
import Concordium.Common.Version
import Concordium.Crypto.SHA256 (DigestSize, Hash (Hash))
import Concordium.Crypto.SignatureScheme (VerifyKey (..))
import Concordium.Crypto.SignatureScheme (VerifyKey (..), Signature(..))
import Concordium.Types.Accounts.Releases
import Concordium.Types.Execution
import qualified Concordium.Wasm as Wasm
Expand Down Expand Up @@ -1527,6 +1528,88 @@ instance ToProto QueryTypes.BakerSummary where
ProtoFields.account .= toProto bakerAccount
ProtoFields.lotteryPower .= bsBakerLotteryPower

instance ToProto Transactions.TransactionHeader where
type Output Transactions.TransactionHeader = Proto.AccountTransactionHeader

toProto Transactions.TransactionHeader{..} = Proto.make $ do
ProtoFields.sender .= toProto thSender
ProtoFields.sequenceNumber .= toProto thNonce
ProtoFields.energyAmount .= toProto thEnergyAmount
ProtoFields.expiry .= toProto thExpiry

instance ToProto Signature where
type Output Signature = Proto.Signature

toProto (Signature bss) = Proto.make $ do
ProtoFields.value .= BSS.fromShort bss

instance ToProto Transactions.TransactionSignature where
type Output Transactions.TransactionSignature = Proto.AccountTransactionSignature

toProto Transactions.TransactionSignature{..} = Proto.make $ do
ProtoFields.signatures .= (Map.fromAscList . map mk . Map.toAscList $ tsSignatures)
where mk (k, s) = (fromIntegral k, mkSingleSig s)
mkSingleSig sigs = Proto.make $ do
ProtoFields.signatures .= (Map.fromAscList . map (\(ki, sig) -> (fromIntegral ki, toProto sig)) . Map.toAscList $ sigs)

instance ToProto Transactions.AccountTransaction where
type Output Transactions.AccountTransaction = Proto.AccountTransaction

toProto Transactions.AccountTransaction{..} = Proto.make $ do
ProtoFields.signature .= toProto atrSignature
ProtoFields.header .= toProto atrHeader
ProtoFields.payload .= Proto.make (
ProtoFields.rawPayload .= BSS.fromShort (_spayload atrPayload)
)

instance ToProto Transactions.AccountCreation where
type Output Transactions.AccountCreation = Proto.CredentialDeployment

toProto Transactions.AccountCreation{..} = Proto.make $ do
ProtoFields.messageExpiry .= toProto messageExpiry
ProtoFields.rawPayload .= S.encode credential

instance ToProto Updates.UpdateInstructionSignatures where
type Output Updates.UpdateInstructionSignatures = Proto.SignatureMap

toProto Updates.UpdateInstructionSignatures{..} = Proto.make $ do
ProtoFields.signatures .= (Map.fromAscList . map mk . Map.toAscList $ signatures)
where mk (k, s) = (fromIntegral k, toProto s)

instance ToProto Updates.UpdateHeader where
type Output Updates.UpdateHeader = Proto.UpdateInstructionHeader

toProto Updates.UpdateHeader{..} = Proto.make $ do
-- since UpdateSequenceNumber is an alias for Nonce in Haskell, but not in
-- the .proto file we have to use mkWord64 or similar, and not toProto since
-- that one is defined for the Nonce.
ProtoFields.sequenceNumber .= mkWord64 updateSeqNumber
ProtoFields.effectiveTime .= toProto updateEffectiveTime
ProtoFields.timeout .= toProto updateTimeout

instance ToProto Updates.UpdateInstruction where
type Output Updates.UpdateInstruction = Proto.UpdateInstruction

toProto Updates.UpdateInstruction{..} = Proto.make $ do
ProtoFields.signatures .= toProto uiSignatures
ProtoFields.header .= toProto uiHeader
ProtoFields.payload .= Proto.make (
ProtoFields.rawPayload .= S.runPut (Updates.putUpdatePayload uiPayload)
)


instance ToProto Transactions.BlockItem where
type Output Transactions.BlockItem = Proto.BlockItem
toProto bi = Proto.make $ do
ProtoFields.hash .= toProto (Transactions.wmdHash bi)
case Transactions.wmdData bi of
Transactions.NormalTransaction accTx -> do
ProtoFields.accountTransaction .= toProto accTx
Transactions.CredentialDeployment cred ->
ProtoFields.credentialDeployment .= toProto cred
Transactions.ChainUpdate cu ->
ProtoFields.updateInstruction .= toProto cu

instance ToProto TxTypes.AccountAmounts where
type Output TxTypes.AccountAmounts = Proto.BlockSpecialEvent'AccountAmounts
toProto TxTypes.AccountAmounts{..} = Proto.make $ ProtoFields.entries .= fmap mapper (Map.toList accountAmounts)
Expand Down Expand Up @@ -1631,10 +1714,10 @@ instance ToProto CredentialsPerBlockLimit where
type Output CredentialsPerBlockLimit = Proto.CredentialsPerBlockLimit
toProto = mkWord16

instance ToProto (AccountAddress, Parameters.EChainParameters) where
type Output (AccountAddress, Parameters.EChainParameters) = Proto.ChainParameters
instance ToProto (AccountAddress, Q.EChainParametersAndKeys) where
type Output (AccountAddress, Q.EChainParametersAndKeys) = Proto.ChainParameters

toProto (foundationAddr, Parameters.EChainParameters (params :: Parameters.ChainParameters' cpv)) =
toProto (foundationAddr, Q.EChainParametersAndKeys (params :: Parameters.ChainParameters' cpv) keys) =
case chainParametersVersion @cpv of
SCPV0 ->
let Parameters.ChainParameters{_cpCooldownParameters = Parameters.CooldownParametersV0 epochs,
Expand All @@ -1654,6 +1737,9 @@ instance ToProto (AccountAddress, Parameters.EChainParameters) where
ProtoFields.gasRewards .= toProto (Parameters._rpGASRewards _cpRewardParameters)
ProtoFields.foundationAccount .= toProto foundationAddr
ProtoFields.minimumThresholdForBaking .= toProto minThreshold
ProtoFields.rootKeys .= toProto (Updates.rootKeys keys)
ProtoFields.level1Keys .= toProto (Updates.level1Keys keys)
ProtoFields.level2Keys .= toProto (Updates.level2Keys keys)
)
SCPV1 ->
let Parameters.ChainParameters{..} = params
Expand All @@ -1672,6 +1758,9 @@ instance ToProto (AccountAddress, Parameters.EChainParameters) where
ProtoFields.gasRewards .= toProto (Parameters._rpGASRewards _cpRewardParameters)
ProtoFields.foundationAccount .= toProto foundationAddr
ProtoFields.poolParameters .= toProto _cpPoolParameters
ProtoFields.rootKeys .= toProto (Updates.rootKeys keys)
ProtoFields.level1Keys .= toProto (Updates.level1Keys keys)
ProtoFields.level2Keys .= toProto (Updates.level2Keys keys)
)

instance ToProto FinalizationIndex where
Expand Down Expand Up @@ -2448,6 +2537,31 @@ getAccountNonFinalizedTransactionsV2 cptr channel accPtr cbk = do
_ <- enqueueMessages (sender channel) res
return (queryResultCode QRSuccess)


-- |Get the block items for a block.
getBlockItemsV2 ::
StablePtr Ext.ConsensusRunner ->
Ptr SenderChannel ->
-- |Block type
Word8 ->
-- |Block hash ptr.
Ptr Word8 ->
-- |Out pointer for writing the block hash that was used.
Ptr Word8 ->
FunPtr ChannelSendCallback ->
IO Int64
getBlockItemsV2 cptr channel blockType blockHashPtr outHash cbk = do
Ext.ConsensusRunner mvr <- deRefStablePtr cptr
let sender = callChannelSendCallback cbk
bhi <- decodeBlockHashInput blockType blockHashPtr
(bh, mBis) <- runMVR (Q.getBlockItems bhi) mvr
case mBis of
Nothing -> return (queryResultCode QRNotFound)
Just items -> do
copyHashTo outHash bh
_ <- enqueueMessages (sender channel) items
return (queryResultCode QRSuccess)

getBlockTransactionEventsV2 ::
StablePtr Ext.ConsensusRunner ->
Ptr SenderChannel ->
Expand Down Expand Up @@ -3041,6 +3155,19 @@ foreign export ccall
FunPtr ChannelSendCallback ->
IO Int64

foreign export ccall
getBlockItemsV2 ::
StablePtr Ext.ConsensusRunner ->
Ptr SenderChannel ->
-- |Block type.
Word8 ->
-- |Block hash.
Ptr Word8 ->
-- |Out pointer for writing the block hash that was used.
Ptr Word8 ->
FunPtr ChannelSendCallback ->
IO Int64

foreign export ccall
getBlockTransactionEventsV2 ::
StablePtr Ext.ConsensusRunner ->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -455,3 +455,4 @@ instance (bs ~ BlockState m, BS.BlockStateStorage m, Monad m, MonadIO m, MonadSt
return $ getNonFinalizedVerificationResult bi table

storeFinalState bs = nextGenesisInitialState ?= bs

24 changes: 19 additions & 5 deletions concordium-consensus/src/Concordium/Queries.hs
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{-# LANGUAGE DataKinds #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeApplications #-}
Expand Down Expand Up @@ -32,8 +33,9 @@ import Concordium.Types.IdentityProviders
import Concordium.Types.Parameters
import Concordium.Types.Queries
import Concordium.Types.SeedState
import Concordium.Types.Transactions
import Concordium.Types.Execution (TransactionSummary)
import Concordium.Types.Transactions (SpecialTransactionOutcome)
import Concordium.Types.Updates
import qualified Concordium.Types.UpdateQueues as UQ
import qualified Concordium.Wasm as Wasm
import qualified Concordium.GlobalState.ContractStateV1 as StateV1
Expand Down Expand Up @@ -62,7 +64,7 @@ import Concordium.Kontrol.BestBlock
import Concordium.MultiVersion
import Concordium.Skov as Skov (
SkovQueryMonad (getBlocksAtHeight),
evalSkovT,
evalSkovT
)

-- |Input to block based queries, i.e., queries which query the state of an
Expand Down Expand Up @@ -502,6 +504,11 @@ getBlockSummary = liftSkovQueryBlock getBlockSummarySkovM
let bsProtocolVersion = protocolVersion @pv
return BlockSummary{..}


-- |Get the block items of a block.
getBlockItems :: forall gsconf finconf. BlockHashInput -> MVR gsconf finconf (BlockHash, Maybe [BlockItem])
getBlockItems = liftSkovQueryBHI (return . blockTransactions)

-- |Get the transaction outcomes in the block.
getBlockTransactionSummaries :: forall gsconf finconf. BlockHashInput -> MVR gsconf finconf (BlockHash, Maybe (Vec.Vector TransactionSummary))
getBlockTransactionSummaries = liftSkovQueryBHI $ BS.getOutcomes <=< blockState
Expand Down Expand Up @@ -587,15 +594,22 @@ getBlockPendingUpdates = liftSkovQueryBHI query
Just (_, acc) -> do
(t, ) . PUEFoundationAccount <$> BS.getAccountCanonicalAddress acc

-- |An existentially qualified pair of chain parameters and update keys currently in effect.
data EChainParametersAndKeys = forall (cpv :: ChainParametersVersion). IsChainParametersVersion cpv =>
EChainParametersAndKeys
{ ecpParams :: !(ChainParameters' cpv)
, ecpKeys :: !(UpdateKeysCollection cpv)
}

-- |Get the chain parameters valid at the end of a given block, as well as the address of the foundation account.
-- The chain parameters contain only the account index of the foundation account.
getBlockChainParameters :: forall gsconf finconf. BlockHashInput -> MVR gsconf finconf (BlockHash, Maybe (AccountAddress, EChainParameters))
getBlockChainParameters :: forall gsconf finconf. BlockHashInput -> MVR gsconf finconf (BlockHash, Maybe (AccountAddress, EChainParametersAndKeys))
getBlockChainParameters = liftSkovQueryBHI query
where
query :: forall pv.
SkovMonad (VersionedSkovM gsconf finconf pv) =>
BlockPointerType (VersionedSkovM gsconf finconf pv) ->
VersionedSkovM gsconf finconf pv (AccountAddress, EChainParameters)
VersionedSkovM gsconf finconf pv (AccountAddress, EChainParametersAndKeys)
query bp = do
bs <- blockState bp
updates <- BS.getUpdates bs
Expand All @@ -604,7 +618,7 @@ getBlockChainParameters = liftSkovQueryBHI query
Nothing -> error "Invariant violation. Foundation account index does not exist in the account table."
Just (_, acc) -> do
foundationAddr <- BS.getAccountCanonicalAddress acc
return (foundationAddr, EChainParameters params)
return (foundationAddr, EChainParametersAndKeys params (_unhashed (UQ._currentKeyCollection updates)))

-- |Get the finalization record contained in the given block, if any.
getBlockFinalizationSummary :: forall gsconf finconf. BlockHashInput -> MVR gsconf finconf (BlockHash, Maybe BlockFinalizationSummary)
Expand Down
2 changes: 1 addition & 1 deletion concordium-grpc-api
10 changes: 10 additions & 0 deletions concordium-node/build.rs
Original file line number Diff line number Diff line change
Expand Up @@ -620,6 +620,16 @@ fn build_grpc2(proto_root_input: &str) -> std::io::Result<()> {
.codec_path("tonic::codec::ProstCodec")
.build(),
)
.method(
tonic_build::manual::Method::builder()
.name("get_block_items")
.route_name("GetBlockItems")
.input_type("crate::grpc2::types::BlockHashInput")
.output_type("Vec<u8>")
.codec_path("crate::grpc2::RawCodec")
.server_streaming()
.build(),
)
.build();
// Due to the slightly hacky nature of the RawCodec (i.e., it does not support
// deserialization) we cannot build the client. But we also don't need it in the
Expand Down
49 changes: 49 additions & 0 deletions concordium-node/src/consensus_ffi/ffi.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1141,6 +1141,28 @@ extern "C" {
) -> i32,
) -> i64;

/// Get a list of items for a given block.
/// The stream will end when all the transactions
/// for the block have been returned
///
/// * `consensus` - Pointer to the current consensus.
/// * `stream` - Pointer to the response stream.
/// * `block_id_type` whether to query latest finalized or a specific block.
/// * `block_hash_ptr` - Pointer to block hash. Must contain 32 bytes.
/// * `callback` - Callback for writing to the response stream.
pub fn getBlockItemsV2(
consensus: *mut consensus_runner,
stream: *mut futures::channel::mpsc::Sender<Result<Vec<u8>, tonic::Status>>,
block_id_type: u8,
block_hash_ptr: *const u8,
out_hash: *mut u8,
callback: extern "C" fn(
*mut futures::channel::mpsc::Sender<Result<Vec<u8>, tonic::Status>>,
*const u8,
i64,
) -> i32,
) -> i64;

/// Get a list of transaction events in a given block.
/// The stream will end when all the transaction events for a given block
/// have been returned.
Expand Down Expand Up @@ -2611,6 +2633,33 @@ impl ConsensusContainer {
Ok(())
}

/// Get a list of block items in a block specified by a block hash.
pub fn get_block_items_v2(
&self,
request: &crate::grpc2::types::BlockHashInput,
sender: futures::channel::mpsc::Sender<Result<Vec<u8>, tonic::Status>>,
) -> Result<[u8; 32], tonic::Status> {
use crate::grpc2::Require;
let consensus = self.consensus.load(Ordering::SeqCst);
let sender = Box::new(sender);
let (block_id_type, block_hash) =
crate::grpc2::types::block_hash_input_to_ffi(request).require()?;
let mut buf = [0u8; 32];
let response: ConsensusQueryResponse = unsafe {
getBlockItemsV2(
consensus,
Box::into_raw(sender),
block_id_type,
block_hash,
buf.as_mut_ptr(),
enqueue_bytearray_callback,
)
}
.try_into()?;
response.ensure_ok("block")?;
Ok(buf)
}

/// Get a list of transaction events in a given block.
/// The stream will end when all the transaction events for a given block
/// have been returned.
Expand Down
13 changes: 13 additions & 0 deletions concordium-node/src/grpc2.rs
Original file line number Diff line number Diff line change
Expand Up @@ -904,6 +904,8 @@ pub mod server {
futures::channel::mpsc::Receiver<Result<Vec<u8>, tonic::Status>>;
/// Return type for the 'GetBakerList' method.
type GetBakerListStream = futures::channel::mpsc::Receiver<Result<Vec<u8>, tonic::Status>>;
/// Return type for `GetBlockItems`.
type GetBlockItemsStream = futures::channel::mpsc::Receiver<Result<Vec<u8>, tonic::Status>>;
/// Return type for the 'GetBlockPendingUpdates' method.
type GetBlockPendingUpdatesStream =
futures::channel::mpsc::Receiver<Result<Vec<u8>, tonic::Status>>;
Expand Down Expand Up @@ -1919,6 +1921,17 @@ pub mod server {
value: sign_hash.to_vec(),
}))
}

async fn get_block_items(
&self,
request: tonic::Request<crate::grpc2::types::BlockHashInput>,
) -> Result<tonic::Response<Self::GetBlockItemsStream>, tonic::Status> {
let (sender, receiver) = futures::channel::mpsc::channel(100);
let hash = self.consensus.get_block_items_v2(request.get_ref(), sender)?;
let mut response = tonic::Response::new(receiver);
add_hash(&mut response, hash)?;
Ok(response)
}
}
}

Expand Down

0 comments on commit 6be4de4

Please sign in to comment.