Skip to content

Commit

Permalink
Merge pull request #3038 from oasisprotocol/kostko/feature/reg-consrp…
Browse files Browse the repository at this point in the history
…c-node-status

Update TLSAddressRequiredRoles and update node control API
  • Loading branch information
kostko authored Jun 22, 2020
2 parents adbf222 + f27819b commit 5455221
Show file tree
Hide file tree
Showing 15 changed files with 217 additions and 45 deletions.
1 change: 1 addition & 0 deletions .changelog/3038.breaking.md
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
go/registry: Make TLS address required for RoleConsensusRPC
5 changes: 5 additions & 0 deletions .changelog/3038.feature.1.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
go/control: Add registration status to node status

This updates the response returned by the `GetStatus` method exposed by the
node controller service to include a `Registration` field that contains
information about the node's current registration.
1 change: 1 addition & 0 deletions .changelog/3038.feature.2.md
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
go/consensus: Add IsValidator to reported node status
6 changes: 6 additions & 0 deletions .changelog/3038.feature.3.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
go/control: Add identity status to node status

This updates the response returned by the `GetStatus` method exposed by the
node controller service to include an `Identity` field that contains
information about the public keys used to identify a node in different
contexts.
3 changes: 3 additions & 0 deletions go/consensus/api/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,9 @@ type Status struct {
GenesisHeight int64 `json:"genesis_height"`
// GenesisHash is the hash of the genesis block.
GenesisHash []byte `json:"genesis_hash"`

// IsValidator returns whether the current node is part of the validator set.
IsValidator bool `json:"is_validator"`
}

// Backend is an interface that a consensus backend must provide.
Expand Down
52 changes: 36 additions & 16 deletions go/consensus/tendermint/tendermint.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
tmproxy "github.com/tendermint/tendermint/proxy"
tmcli "github.com/tendermint/tendermint/rpc/client/local"
tmrpctypes "github.com/tendermint/tendermint/rpc/core/types"
tmstate "github.com/tendermint/tendermint/state"
tmtypes "github.com/tendermint/tendermint/types"
tmdb "github.com/tendermint/tm-db"

Expand Down Expand Up @@ -719,38 +720,57 @@ func (t *tendermintService) GetTransactions(ctx context.Context, height int64) (
}

func (t *tendermintService) GetStatus(ctx context.Context) (*consensusAPI.Status, error) {
status := &consensusAPI.Status{
ConsensusVersion: version.ConsensusProtocol.String(),
Backend: api.BackendName,
}

// Genesis block is hardcoded as block 1, since tendermint doesn't have
// a genesis block as such, but some external tooling expects there to be
// one, so here we are.
// This may soon change if the following tendermint issue gets fixed:
// https://github.com/tendermint/tendermint/issues/2543
status.GenesisHeight = 1
genBlk, err := t.GetBlock(ctx, 1)
if err != nil {
return nil, err
switch err {
case nil:
status.GenesisHash = genBlk.Hash
default:
// We may not be able to fetch the genesis block in case it has been pruned.
}

// Latest block.
latestBlk, err := t.GetBlock(ctx, consensusAPI.HeightLatest)
if err != nil {
return nil, err
}

switch err {
case nil:
status.LatestHeight = latestBlk.Height
status.LatestHash = latestBlk.Hash
status.LatestTime = latestBlk.Time
case consensusAPI.ErrNoCommittedBlocks:
// No committed blocks yet.
default:
return nil, fmt.Errorf("failed to fetch current block: %w", err)
}

// List of consensus peers.
tmpeers := t.node.Switch().Peers().List()
peers := make([]string, 0, len(tmpeers))
for _, tmpeer := range tmpeers {
p := string(tmpeer.ID()) + "@" + tmpeer.RemoteAddr().String()
peers = append(peers, p)
}
status.NodePeers = peers

return &consensusAPI.Status{
ConsensusVersion: version.ConsensusProtocol.String(),
Backend: api.BackendName,
NodePeers: peers,
LatestHeight: latestBlk.Height,
LatestHash: latestBlk.Hash,
LatestTime: latestBlk.Time,
GenesisHeight: 1, // See above for an explanation why this is 1.
GenesisHash: genBlk.Hash,
}, nil
// Check if the local node is in the validator set for the latest (uncommitted) block.
vals, err := tmstate.LoadValidators(t.stateDb, status.LatestHeight+1)
if err != nil {
return nil, fmt.Errorf("failed to load validator set: %w", err)
}
consensusPk := t.consensusSigner.Public()
consensusAddr := []byte(crypto.PublicKeyToTendermint(&consensusPk).Address())
status.IsValidator = vals.HasAddress(consensusAddr)

return status, nil
}

func (t *tendermintService) WatchBlocks(ctx context.Context) (<-chan *consensusAPI.Block, pubsub.ClosableSubscription, error) {
Expand Down
45 changes: 44 additions & 1 deletion go/control/api/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,12 @@ package api

import (
"context"
"time"

"github.com/oasisprotocol/oasis-core/go/common/crypto/signature"
"github.com/oasisprotocol/oasis-core/go/common/errors"
"github.com/oasisprotocol/oasis-core/go/common/identity"
"github.com/oasisprotocol/oasis-core/go/common/node"
consensus "github.com/oasisprotocol/oasis-core/go/consensus/api"
epochtime "github.com/oasisprotocol/oasis-core/go/epochtime/api"
upgrade "github.com/oasisprotocol/oasis-core/go/upgrade/api"
Expand Down Expand Up @@ -47,17 +51,56 @@ type Status struct {
// SoftwareVersion is the oasis-node software version.
SoftwareVersion string `json:"software_version"`

// Identity is the identity of the node.
Identity IdentityStatus `json:"identity"`

// Consensus is the status overview of the consensus layer.
Consensus consensus.Status `json:"consensus"`

// Registration is the node's registration status.
Registration RegistrationStatus `json:"registration"`
}

// IdentityStatus is the current node identity status, listing all the public keys that identify
// this node in different contexts.
type IdentityStatus struct {
// Node is the node identity public key.
Node signature.PublicKey `json:"node"`

// P2P is the public key used for p2p communication.
P2P signature.PublicKey `json:"p2p"`

// Consensus is the consensus public key.
Consensus signature.PublicKey `json:"consensus"`

// TLS is the public key used for TLS connections.
TLS signature.PublicKey `json:"tls"`
}

// RegistrationStatus is the node registration status.
type RegistrationStatus struct {
// LastRegistration is the time of the last successful registration with the consensus registry
// service. In case the node did not successfully register yet, it will be the zero timestamp.
LastRegistration time.Time `json:"last_registration"`

// Descriptor is the node descriptor that the node successfully registered with. In case the
// node did not successfully register yet, it will be nil.
Descriptor *node.Node `json:"descriptor,omitempty"`
}

// ControlledNode is an interface the node presents for shutting itself down.
// ControlledNode is an internal interface that the controlled oasis-node must provide.
type ControlledNode interface {
// RequestShutdown is the method called by the control server to trigger node shutdown.
RequestShutdown() (<-chan struct{}, error)

// Ready returns a channel that is closed once node is ready.
Ready() <-chan struct{}

// GetIdentity returns the node's identity.
GetIdentity() *identity.Identity

// GetRegistrationStatus returns the node's current registration status.
GetRegistrationStatus(ctx context.Context) (*RegistrationStatus, error)
}

// DebugModuleName is the module name for the debug controller service.
Expand Down
19 changes: 17 additions & 2 deletions go/control/control.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ package control

import (
"context"
"fmt"

"github.com/oasisprotocol/oasis-core/go/common/version"
consensus "github.com/oasisprotocol/oasis-core/go/consensus/api"
Expand Down Expand Up @@ -83,12 +84,26 @@ func (c *nodeController) CancelUpgrade(ctx context.Context) error {
func (c *nodeController) GetStatus(ctx context.Context) (*control.Status, error) {
cs, err := c.consensus.GetStatus(ctx)
if err != nil {
return nil, err
return nil, fmt.Errorf("failed to get consensus status: %w", err)
}

rs, err := c.node.GetRegistrationStatus(ctx)
if err != nil {
return nil, fmt.Errorf("failed to get registration status: %w", err)
}

ident := c.node.GetIdentity()

return &control.Status{
SoftwareVersion: version.SoftwareVersion,
Consensus: *cs,
Identity: control.IdentityStatus{
Node: ident.NodeSigner.Public(),
P2P: ident.P2PSigner.Public(),
Consensus: ident.ConsensusSigner.Public(),
TLS: ident.GetTLSSigner().Public(),
},
Consensus: *cs,
Registration: *rs,
}, nil
}

Expand Down
19 changes: 19 additions & 0 deletions go/oasis-node/cmd/debug/txsource/workload/queries.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/oasisprotocol/oasis-core/go/common/node"
"github.com/oasisprotocol/oasis-core/go/common/quantity"
consensus "github.com/oasisprotocol/oasis-core/go/consensus/api"
control "github.com/oasisprotocol/oasis-core/go/control/api"
epochtime "github.com/oasisprotocol/oasis-core/go/epochtime/api"
registry "github.com/oasisprotocol/oasis-core/go/registry/api"
runtimeClient "github.com/oasisprotocol/oasis-core/go/runtime/client/api"
Expand Down Expand Up @@ -57,6 +58,7 @@ type queries struct {
stakingParams staking.ConsensusParameters
schedulerParams scheduler.ConsensusParameters

control control.NodeController
staking staking.Backend
consensus consensus.ClientBackend
registry registry.Backend
Expand Down Expand Up @@ -439,6 +441,19 @@ func (q *queries) doRuntimeQueries(ctx context.Context, rng *rand.Rand) error {
return nil
}

func (q *queries) doControlQueries(ctx context.Context, rng *rand.Rand) error {
q.logger.Debug("Doing node control queries")

_, err := q.control.GetStatus(ctx)
if err != nil {
return fmt.Errorf("control.GetStatus error: %w", err)
}

q.logger.Debug("Done node control queries")

return nil
}

func (q *queries) doQueries(ctx context.Context, rng *rand.Rand) error {
block, err := q.consensus.GetBlock(ctx, consensus.HeightLatest)
if err != nil {
Expand Down Expand Up @@ -470,6 +485,9 @@ func (q *queries) doQueries(ctx context.Context, rng *rand.Rand) error {
"height_latest", block.Height,
)

if err := q.doControlQueries(ctx, rng); err != nil {
return fmt.Errorf("control queries error: %w", err)
}
if err := q.doConsensusQueries(ctx, rng, height); err != nil {
return fmt.Errorf("consensus queries error: %w", err)
}
Expand Down Expand Up @@ -499,6 +517,7 @@ func (q *queries) Run(gracefulExit context.Context, rng *rand.Rand, conn *grpc.C

q.logger = logging.GetLogger("cmd/txsource/workload/queries")

q.control = control.NewNodeControllerClient(conn)
q.consensus = cnsc
q.registry = registry.NewRegistryClient(conn)
q.runtime = runtimeClient.NewRuntimeClient(conn)
Expand Down
37 changes: 37 additions & 0 deletions go/oasis-node/cmd/node/control.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
package node

import (
"context"

"github.com/oasisprotocol/oasis-core/go/common/identity"
control "github.com/oasisprotocol/oasis-core/go/control/api"
)

var _ control.ControlledNode = (*Node)(nil)

// Implements control.ControlledNode.
func (n *Node) RequestShutdown() (<-chan struct{}, error) {
if err := n.RegistrationWorker.RequestDeregistration(); err != nil {
return nil, err
}
// This returns only the registration worker's event channel,
// otherwise the caller (usually the control grpc server) will only
// get notified once everything is already torn down - perhaps
// including the server.
return n.RegistrationWorker.Quit(), nil
}

// Implements control.ControlledNode.
func (n *Node) Ready() <-chan struct{} {
return n.readyCh
}

// Implements control.ControlledNode.
func (n *Node) GetIdentity() *identity.Identity {
return n.Identity
}

// Implements control.ControlledNode.
func (n *Node) GetRegistrationStatus(ctx context.Context) (*control.RegistrationStatus, error) {
return n.RegistrationWorker.GetRegistrationStatus(ctx)
}
24 changes: 2 additions & 22 deletions go/oasis-node/cmd/node/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,12 +69,8 @@ import (
workerStorage "github.com/oasisprotocol/oasis-core/go/worker/storage"
)

var (
_ controlAPI.ControlledNode = (*Node)(nil)

// Flags has the configuration flags.
Flags = flag.NewFlagSet("", flag.ContinueOnError)
)
// Flags has the configuration flags.
var Flags = flag.NewFlagSet("", flag.ContinueOnError)

const exportsSubDir = "exports"

Expand Down Expand Up @@ -161,22 +157,6 @@ func (n *Node) Wait() {
n.svcMgr.Wait()
}

func (n *Node) RequestShutdown() (<-chan struct{}, error) {
if err := n.RegistrationWorker.RequestDeregistration(); err != nil {
return nil, err
}
// This returns only the registration worker's event channel,
// otherwise the caller (usually the control grpc server) will only
// get notified once everything is already torn down - perhaps
// including the server.
return n.RegistrationWorker.Quit(), nil
}

// Ready returns the ready channel which gets closed once the node is ready.
func (n *Node) Ready() <-chan struct{} {
return n.readyCh
}

func (n *Node) waitReady(logger *logging.Logger) {
if n.NodeController == nil {
logger.Error("failed while waiting for node: node controller not initialized")
Expand Down
12 changes: 12 additions & 0 deletions go/oasis-test-runner/scenario/e2e/early_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,5 +76,17 @@ func (sc *earlyQueryImpl) Run(childEnv *env.Env) error {
return fmt.Errorf("GetTransactions query should fail with ErrNoCommittedBlocks (got: %s)", err)
}

// GetStatus.
status, err := sc.net.Controller().GetStatus(ctx)
if err != nil {
return fmt.Errorf("failed to get status for node: %w", err)
}
if status.Consensus.LatestHeight != 0 {
return fmt.Errorf("node reports non-zero latest height before chain is initialized")
}
if !status.Consensus.IsValidator {
return fmt.Errorf("node does not report itself to be a validator at genesis")
}

return nil
}
14 changes: 12 additions & 2 deletions go/oasis-test-runner/scenario/e2e/node_shutdown.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,18 +50,28 @@ func (sc *nodeShutdownImpl) Run(childEnv *env.Env) error {
return err
}

sc.logger.Info("requesting node shutdown")
sc.logger.Info("waiting for the node to become ready")
computeWorker := sc.runtimeImpl.net.ComputeWorkers()[0]

// Wait for the node to be ready since we didn't wait for any clients.
nodeCtrl, err := oasis.NewController(computeWorker.SocketPath())
if err != nil {
return err
}
if err = nodeCtrl.WaitSync(context.Background()); err != nil {
if err = nodeCtrl.WaitReady(context.Background()); err != nil {
return err
}

// Make sure that the GetStatus endpoint returns sensible values.
status, err := nodeCtrl.GetStatus(context.Background())
if err != nil {
return fmt.Errorf("failed to get status for node: %w", err)
}
if status.Registration.Descriptor == nil {
return fmt.Errorf("node has not registered")
}

sc.logger.Info("requesting node shutdown")
args := []string{
"control", "shutdown",
"--log.level", "debug",
Expand Down
Loading

0 comments on commit 5455221

Please sign in to comment.