Skip to content

Commit

Permalink
Add static analysis for unsafe uint casting (prysmaticlabs#10318)
Browse files Browse the repository at this point in the history
* Add static analysis for unsafe uint casting

* Fix violations of uintcast

* go mod tidy

* Add exclusion to nogo for darwin build

* Add test for math.Int

* Move some things to const so they are assured not to exceed int64

* Self review

* lint

* fix tests

* fix test

* Add init check for non 64 bit OS

* Move new deps from WORKSPACE to deps.bzl

* fix bazel build for go analysis runs

* Update BUILD.bazel

Remove TODO

* add math.AddInt method

* Add new test casts

* Add case where builtin functions and declared functions are covered

* Fix new findings

* cleanup

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
Co-authored-by: Nishant Das <[email protected]>
  • Loading branch information
3 people authored Mar 11, 2022
1 parent 693cc79 commit c1197d7
Show file tree
Hide file tree
Showing 99 changed files with 1,073 additions and 212 deletions.
3 changes: 3 additions & 0 deletions .bazelrc
Original file line number Diff line number Diff line change
Expand Up @@ -217,3 +217,6 @@ build:remote --remote_local_fallback

# Ignore GoStdLib with remote caching
build --modify_execution_info='GoStdlib.*=+no-remote-cache'

# Set bazel gotag
build --define gotags=bazel
3 changes: 3 additions & 0 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ exports_files([
# gazelle:map_kind go_library go_library @prysm//tools/go:def.bzl
# gazelle:map_kind go_test go_test @prysm//tools/go:def.bzl
# gazelle:map_kind go_repository go_repository @prysm//tools/go:def.bzl
# gazelle:build_tags bazel
# gazelle:exclude tools/analyzers/**/testdata/**
gazelle(
name = "gazelle",
prefix = prefix,
Expand Down Expand Up @@ -125,6 +127,7 @@ nogo(
"//tools/analyzers/ineffassign:go_default_library",
"//tools/analyzers/properpermissions:go_default_library",
"//tools/analyzers/recursivelock:go_default_library",
"//tools/analyzers/uintcast:go_default_library",
] + select({
# nogo checks that fail with coverage enabled.
":coverage_enabled": [],
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/blockchain/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ func (s *Service) Status() error {

func (s *Service) startFromSavedState(saved state.BeaconState) error {
log.Info("Blockchain data already exists in DB, initializing...")
s.genesisTime = time.Unix(int64(saved.GenesisTime()), 0)
s.genesisTime = time.Unix(int64(saved.GenesisTime()), 0) // lint:ignore uintcast -- Genesis time will not exceed int64 in your lifetime.
s.cfg.AttService.SetGenesisTime(saved.GenesisTime())

originRoot, err := s.originRootFromSavedState(s.ctx)
Expand Down
9 changes: 6 additions & 3 deletions beacon-chain/cache/active_balance.go
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
//go:build !fuzz
// +build !fuzz

package cache
Expand All @@ -17,10 +18,12 @@ import (
"github.com/prysmaticlabs/prysm/config/params"
)

var (
const (
// maxBalanceCacheSize defines the max number of active balances can cache.
maxBalanceCacheSize = uint64(4)
maxBalanceCacheSize = int(4)
)

var (
// BalanceCacheMiss tracks the number of balance requests that aren't present in the cache.
balanceCacheMiss = promauto.NewCounter(prometheus.CounterOpts{
Name: "total_effective_balance_cache_miss",
Expand All @@ -42,7 +45,7 @@ type BalanceCache struct {
// NewEffectiveBalanceCache creates a new effective balance cache for storing/accessing total balance by epoch.
func NewEffectiveBalanceCache() *BalanceCache {
return &BalanceCache{
cache: lruwrpr.New(int(maxBalanceCacheSize)),
cache: lruwrpr.New(maxBalanceCacheSize),
}
}

Expand Down
9 changes: 6 additions & 3 deletions beacon-chain/cache/committee.go
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
//go:build !fuzz
// +build !fuzz

package cache
Expand All @@ -19,11 +20,13 @@ import (
mathutil "github.com/prysmaticlabs/prysm/math"
)

var (
const (
// maxCommitteesCacheSize defines the max number of shuffled committees on per randao basis can cache.
// Due to reorgs and long finality, it's good to keep the old cache around for quickly switch over.
maxCommitteesCacheSize = uint64(32)
maxCommitteesCacheSize = int(32)
)

var (
// CommitteeCacheMiss tracks the number of committee requests that aren't present in the cache.
CommitteeCacheMiss = promauto.NewCounter(prometheus.CounterOpts{
Name: "committee_cache_miss",
Expand Down Expand Up @@ -55,7 +58,7 @@ func committeeKeyFn(obj interface{}) (string, error) {
// NewCommitteesCache creates a new committee cache for storing/accessing shuffled indices of a committee.
func NewCommitteesCache() *CommitteeCache {
return &CommitteeCache{
CommitteeCache: lruwrpr.New(int(maxCommitteesCacheSize)),
CommitteeCache: lruwrpr.New(maxCommitteesCacheSize),
inProgress: make(map[string]bool),
}
}
Expand Down
4 changes: 2 additions & 2 deletions beacon-chain/cache/committee_fuzz_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func TestCommitteeCache_FuzzCommitteesByEpoch(t *testing.T) {
require.NoError(t, err)
}

assert.Equal(t, maxCommitteesCacheSize, uint64(len(cache.CommitteeCache.Keys())), "Incorrect key size")
assert.Equal(t, maxCommitteesCacheSize, len(cache.CommitteeCache.Keys()), "Incorrect key size")
}

func TestCommitteeCache_FuzzActiveIndices(t *testing.T) {
Expand All @@ -50,5 +50,5 @@ func TestCommitteeCache_FuzzActiveIndices(t *testing.T) {
assert.DeepEqual(t, c.SortedIndices, indices)
}

assert.Equal(t, maxCommitteesCacheSize, uint64(len(cache.CommitteeCache.Keys())), "Incorrect key size")
assert.Equal(t, maxCommitteesCacheSize, len(cache.CommitteeCache.Keys()), "Incorrect key size")
}
2 changes: 1 addition & 1 deletion beacon-chain/cache/committee_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func TestCommitteeCache_CanRotate(t *testing.T) {
}

k := cache.CommitteeCache.Keys()
assert.Equal(t, maxCommitteesCacheSize, uint64(len(k)))
assert.Equal(t, maxCommitteesCacheSize, len(k))

sort.Slice(k, func(i, j int) bool {
return k[i].(string) < k[j].(string)
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/cache/subnet_ids.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ var SubnetIDs = newSubnetIDs()
func newSubnetIDs() *subnetIDs {
// Given a node can calculate committee assignments of current epoch and next epoch.
// Max size is set to 2 epoch length.
cacheSize := int(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().MaxCommitteesPerSlot * 2))
cacheSize := int(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().MaxCommitteesPerSlot * 2)) // lint:ignore uintcast -- constant values that would panic on startup if negative.
attesterCache := lruwrpr.New(cacheSize)
aggregatorCache := lruwrpr.New(cacheSize)
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/core/blocks/deposit.go
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ func verifyDeposit(beaconState state.ReadOnlyBeaconState, deposit *ethpb.Deposit
if ok := trie.VerifyMerkleProofWithDepth(
receiptRoot,
leaf[:],
int(beaconState.Eth1DepositIndex()),
beaconState.Eth1DepositIndex(),
deposit.Proof,
params.BeaconConfig().DepositContractTreeDepth,
); !ok {
Expand Down
6 changes: 3 additions & 3 deletions beacon-chain/core/transition/transition.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func ExecuteStateTransition(
func ProcessSlot(ctx context.Context, state state.BeaconState) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "core.state.ProcessSlot")
defer span.End()
span.AddAttributes(trace.Int64Attribute("slot", int64(state.Slot())))
span.AddAttributes(trace.Int64Attribute("slot", int64(state.Slot()))) // lint:ignore uintcast -- This is OK for tracing.

prevStateRoot, err := state.HashTreeRoot(ctx)
if err != nil {
Expand Down Expand Up @@ -190,7 +190,7 @@ func ProcessSlots(ctx context.Context, state state.BeaconState, slot types.Slot)
if state == nil || state.IsNil() {
return nil, errors.New("nil state")
}
span.AddAttributes(trace.Int64Attribute("slots", int64(slot)-int64(state.Slot())))
span.AddAttributes(trace.Int64Attribute("slots", int64(slot)-int64(state.Slot()))) // lint:ignore uintcast -- This is OK for tracing.

// The block must have a higher slot than parent state.
if state.Slot() >= slot {
Expand Down Expand Up @@ -354,7 +354,7 @@ func VerifyOperationLengths(_ context.Context, state state.BeaconState, b block.
func ProcessEpochPrecompute(ctx context.Context, state state.BeaconState) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "core.state.ProcessEpochPrecompute")
defer span.End()
span.AddAttributes(trace.Int64Attribute("epoch", int64(time.CurrentEpoch(state))))
span.AddAttributes(trace.Int64Attribute("epoch", int64(time.CurrentEpoch(state)))) // lint:ignore uintcast -- This is OK for tracing.

if state == nil || state.IsNil() {
return nil, errors.New("nil state")
Expand Down
1 change: 1 addition & 0 deletions beacon-chain/forkchoice/protoarray/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
deps = [
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//math:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",
Expand Down
17 changes: 13 additions & 4 deletions beacon-chain/forkchoice/protoarray/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"context"

"github.com/prysmaticlabs/prysm/config/params"
pmath "github.com/prysmaticlabs/prysm/math"
"go.opencensus.io/trace"
)

Expand Down Expand Up @@ -46,19 +47,27 @@ func computeDeltas(
if ok {
// Protection against out of bound, the `nextDeltaIndex` which defines
// the block location in the dag can not exceed the total `delta` length.
if int(nextDeltaIndex) >= len(deltas) {
if nextDeltaIndex >= uint64(len(deltas)) {
return nil, nil, errInvalidNodeDelta
}
deltas[nextDeltaIndex] += int(newBalance)
delta, err := pmath.Int(newBalance)
if err != nil {
return nil, nil, err
}
deltas[nextDeltaIndex] += delta
}

currentDeltaIndex, ok := blockIndices[vote.currentRoot]
if ok {
// Protection against out of bound (same as above)
if int(currentDeltaIndex) >= len(deltas) {
if currentDeltaIndex >= uint64(len(deltas)) {
return nil, nil, errInvalidNodeDelta
}
deltas[currentDeltaIndex] -= int(oldBalance)
delta, err := pmath.Int(oldBalance)
if err != nil {
return nil, nil, err
}
deltas[currentDeltaIndex] -= delta
}
}

Expand Down
7 changes: 6 additions & 1 deletion beacon-chain/forkchoice/protoarray/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
pmath "github.com/prysmaticlabs/prysm/math"
pbrpc "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"go.opencensus.io/trace"
)
Expand Down Expand Up @@ -446,7 +447,11 @@ func (s *Store) applyWeightChanges(
s.proposerBoostLock.Unlock()
return err
}
nodeDelta = nodeDelta + int(proposerScore)
iProposerScore, err := pmath.Int(proposerScore)
if err != nil {
return err
}
nodeDelta = nodeDelta + iProposerScore
}
s.proposerBoostLock.Unlock()

Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/node/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ func configureHistoricalSlasher(cliCtx *cli.Context) {
params.OverrideBeaconConfig(c)
cmdConfig := cmd.Get()
// Allow up to 4096 attestations at a time to be requested from the beacon nde.
cmdConfig.MaxRPCPageSize = int(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().MaxAttestations))
cmdConfig.MaxRPCPageSize = int(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().MaxAttestations)) // lint:ignore uintcast -- Page size should not exceed int64 with these constants.
cmd.Init(cmdConfig)
log.Warnf(
"Setting %d slots per archive point and %d max RPC page size for historical slasher usage. This requires additional storage",
Expand Down
12 changes: 7 additions & 5 deletions beacon-chain/p2p/broadcaster.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,8 +104,8 @@ func (s *Service) broadcastAttestation(ctx context.Context, subnet uint64, att *

span.AddAttributes(
trace.BoolAttribute("hasPeer", hasPeer),
trace.Int64Attribute("slot", int64(att.Data.Slot)),
trace.Int64Attribute("subnet", int64(subnet)),
trace.Int64Attribute("slot", int64(att.Data.Slot)), // lint:ignore uintcast -- It's safe to do this for tracing.
trace.Int64Attribute("subnet", int64(subnet)), // lint:ignore uintcast -- It's safe to do this for tracing.
)

if !hasPeer {
Expand Down Expand Up @@ -160,8 +160,8 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs

span.AddAttributes(
trace.BoolAttribute("hasPeer", hasPeer),
trace.Int64Attribute("slot", int64(sMsg.Slot)),
trace.Int64Attribute("subnet", int64(subnet)),
trace.Int64Attribute("slot", int64(sMsg.Slot)), // lint:ignore uintcast -- It's safe to do this for tracing.
trace.Int64Attribute("subnet", int64(subnet)), // lint:ignore uintcast -- It's safe to do this for tracing.
)

if !hasPeer {
Expand Down Expand Up @@ -213,7 +213,9 @@ func (s *Service) broadcastObject(ctx context.Context, obj ssz.Marshaler, topic
if span.IsRecordingEvents() {
id := hash.FastSum64(buf.Bytes())
messageLen := int64(buf.Len())
span.AddMessageSendEvent(int64(id), messageLen /*uncompressed*/, messageLen /*compressed*/)
// lint:ignore uintcast -- It's safe to do this for tracing.
iid := int64(id)
span.AddMessageSendEvent(iid, messageLen /*uncompressed*/, messageLen /*compressed*/)
}
if err := s.PublishToTopic(ctx, topic+s.Encoding().ProtocolSuffix(), buf.Bytes()); err != nil {
err := errors.Wrap(err, "could not publish message")
Expand Down
1 change: 1 addition & 0 deletions beacon-chain/p2p/encoder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
],
deps = [
"//config/params:go_default_library",
"//math:go_default_library",
"@com_github_ferranbt_fastssz//:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_golang_snappy//:go_default_library",
Expand Down
10 changes: 5 additions & 5 deletions beacon-chain/p2p/encoder/ssz.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,14 +3,14 @@ package encoder
import (
"fmt"
"io"
"math"
"sync"

fastssz "github.com/ferranbt/fastssz"
"github.com/gogo/protobuf/proto"
"github.com/golang/snappy"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/math"
)

var _ NetworkEncoding = (*SszNetworkEncoder)(nil)
Expand Down Expand Up @@ -145,11 +145,11 @@ func (_ SszNetworkEncoder) ProtocolSuffix() string {
// MaxLength specifies the maximum possible length of an encoded
// chunk of data.
func (_ SszNetworkEncoder) MaxLength(length uint64) (int, error) {
// Defensive check to prevent potential issues when casting to int64.
if length > math.MaxInt64 {
return 0, errors.Errorf("invalid length provided: %d", length)
il, err := math.Int(length)
if err != nil {
return 0, errors.Wrap(err, "invalid length provided")
}
maxLen := snappy.MaxEncodedLen(int(length))
maxLen := snappy.MaxEncodedLen(il)
if maxLen < 0 {
return 0, errors.Errorf("max encoded length is negative: %d", maxLen)
}
Expand Down
39 changes: 35 additions & 4 deletions beacon-chain/p2p/message_id.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import (
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/hash"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/math"
"github.com/prysmaticlabs/prysm/network/forks"
)

Expand Down Expand Up @@ -72,8 +73,8 @@ func MsgID(genesisValidatorsRoot []byte, pmsg *pubsub_pb.Message) string {
// the topic byte string, and the raw message data: i.e. SHA256(MESSAGE_DOMAIN_INVALID_SNAPPY + uint_to_bytes(uint64(len(message.topic))) + message.topic + message.data)[:20].
func postAltairMsgID(pmsg *pubsub_pb.Message, fEpoch types.Epoch) string {
topic := *pmsg.Topic
topicLen := uint64(len(topic))
topicLenBytes := bytesutil.Uint64ToBytesLittleEndian(topicLen)
topicLen := len(topic)
topicLenBytes := bytesutil.Uint64ToBytesLittleEndian(uint64(topicLen)) // topicLen cannot be negative

// beyond Bellatrix epoch, allow 10 Mib gossip data size
gossipPubSubSize := params.BeaconNetworkConfig().GossipMaxSize
Expand All @@ -83,7 +84,25 @@ func postAltairMsgID(pmsg *pubsub_pb.Message, fEpoch types.Epoch) string {

decodedData, err := encoder.DecodeSnappy(pmsg.Data, gossipPubSubSize)
if err != nil {
totalLength := len(params.BeaconNetworkConfig().MessageDomainInvalidSnappy) + len(topicLenBytes) + int(topicLen) + len(pmsg.Data)
totalLength, err := math.AddInt(
len(params.BeaconNetworkConfig().MessageDomainValidSnappy),
len(topicLenBytes),
topicLen,
len(pmsg.Data),
)
if err != nil {
log.WithError(err).Error("Failed to sum lengths of message domain and topic")
// should never happen
msg := make([]byte, 20)
copy(msg, "invalid")
return string(msg)
}
if uint64(totalLength) > gossipPubSubSize {
// this should never happen
msg := make([]byte, 20)
copy(msg, "invalid")
return string(msg)
}
combinedData := make([]byte, 0, totalLength)
combinedData = append(combinedData, params.BeaconNetworkConfig().MessageDomainInvalidSnappy[:]...)
combinedData = append(combinedData, topicLenBytes...)
Expand All @@ -92,7 +111,19 @@ func postAltairMsgID(pmsg *pubsub_pb.Message, fEpoch types.Epoch) string {
h := hash.Hash(combinedData)
return string(h[:20])
}
totalLength := len(params.BeaconNetworkConfig().MessageDomainValidSnappy) + len(topicLenBytes) + int(topicLen) + len(decodedData)
totalLength, err := math.AddInt(
len(params.BeaconNetworkConfig().MessageDomainValidSnappy),
len(topicLenBytes),
topicLen,
len(decodedData),
)
if err != nil {
log.WithError(err).Error("Failed to sum lengths of message domain and topic")
// should never happen
msg := make([]byte, 20)
copy(msg, "invalid")
return string(msg)
}
combinedData := make([]byte, 0, totalLength)
combinedData = append(combinedData, params.BeaconNetworkConfig().MessageDomainValidSnappy[:]...)
combinedData = append(combinedData, topicLenBytes...)
Expand Down
Loading

0 comments on commit c1197d7

Please sign in to comment.