Relay all block data + statediffs over full node websocket subscription #8
@ -157,8 +157,14 @@ func makeFullNode(ctx *cli.Context) *node.Node {
|
||||
if ctx.GlobalIsSet(utils.ConstantinopleOverrideFlag.Name) {
|
||||
cfg.Eth.ConstantinopleOverride = new(big.Int).SetUint64(ctx.GlobalUint64(utils.ConstantinopleOverrideFlag.Name))
|
||||
}
|
||||
|
||||
utils.RegisterEthService(stack, &cfg.Eth)
|
||||
|
||||
if ctx.GlobalBool(utils.StateDiffFlag.Name) {
|
||||
cfg.Eth.StateDiff = true
|
||||
utils.RegisterStateDiffService(stack, ctx)
|
||||
}
|
||||
|
||||
if ctx.GlobalBool(utils.DashboardEnabledFlag.Name) {
|
||||
utils.RegisterDashboardService(stack, &cfg.Dashboard, gitCommit)
|
||||
}
|
||||
@ -189,6 +195,7 @@ func makeFullNode(ctx *cli.Context) *node.Node {
|
||||
if cfg.Ethstats.URL != "" {
|
||||
utils.RegisterEthStatsService(stack, cfg.Ethstats.URL)
|
||||
}
|
||||
|
||||
return stack
|
||||
}
|
||||
|
||||
|
@ -142,6 +142,7 @@ var (
|
||||
utils.GpoPercentileFlag,
|
||||
utils.EWASMInterpreterFlag,
|
||||
utils.EVMInterpreterFlag,
|
||||
utils.StateDiffFlag,
|
||||
configFileFlag,
|
||||
}
|
||||
|
||||
|
@ -245,6 +245,12 @@ var AppHelpFlagGroups = []flagGroup{
|
||||
utils.MinerLegacyExtraDataFlag,
|
||||
},
|
||||
},
|
||||
{
|
||||
Name: "STATE DIFF",
|
||||
Flags: []cli.Flag{
|
||||
utils.StateDiffFlag,
|
||||
},
|
||||
},
|
||||
{
|
||||
Name: "MISC",
|
||||
},
|
||||
|
@ -29,6 +29,8 @@ import (
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
cli "gopkg.in/urfave/cli.v1"
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts"
|
||||
"github.com/ethereum/go-ethereum/accounts/keystore"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
@ -56,8 +58,8 @@ import (
|
||||
"github.com/ethereum/go-ethereum/p2p/nat"
|
||||
"github.com/ethereum/go-ethereum/p2p/netutil"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
whisper "github.com/ethereum/go-ethereum/whisper/whisperv6"
|
||||
cli "gopkg.in/urfave/cli.v1"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -707,6 +709,11 @@ var (
|
||||
Usage: "External EVM configuration (default = built-in interpreter)",
|
||||
Value: "",
|
||||
}
|
||||
|
||||
StateDiffFlag = cli.BoolFlag{
|
||||
Name: "statediff",
|
||||
Usage: "Enables the calculation of state diffs between each block, persists these state diffs the configured persistence mode.",
|
||||
}
|
||||
)
|
||||
|
||||
// MakeDataDir retrieves the currently requested data directory, terminating
|
||||
@ -918,6 +925,9 @@ func setWS(ctx *cli.Context, cfg *node.Config) {
|
||||
if ctx.GlobalIsSet(WSApiFlag.Name) {
|
||||
cfg.WSModules = splitAndTrim(ctx.GlobalString(WSApiFlag.Name))
|
||||
}
|
||||
if ctx.GlobalBool(StateDiffFlag.Name) {
|
||||
cfg.WSModules = append(cfg.WSModules, "statediff")
|
||||
}
|
||||
}
|
||||
|
||||
// setIPC creates an IPC path configuration from the set command line flags,
|
||||
@ -1516,6 +1526,19 @@ func RegisterEthStatsService(stack *node.Node, url string) {
|
||||
}
|
||||
}
|
||||
|
||||
// RegisterStateDiffService configures and registers a service to stream state diff data over RPC
|
||||
func RegisterStateDiffService(stack *node.Node, ctx *cli.Context) {
|
||||
if err := stack.Register(func(ctx *node.ServiceContext) (node.Service, error) {
|
||||
var ethServ *eth.Ethereum
|
||||
ctx.Service(ðServ)
|
||||
chainDb := ethServ.ChainDb()
|
||||
blockChain := ethServ.BlockChain()
|
||||
return statediff.NewStateDiffService(chainDb, blockChain)
|
||||
}); err != nil {
|
||||
Fatalf("Failed to register State Diff Service", err)
|
||||
}
|
||||
}
|
||||
|
||||
func SetupMetrics(ctx *cli.Context) {
|
||||
if metrics.Enabled {
|
||||
log.Info("Enabling metrics collection")
|
||||
|
@ -90,11 +90,12 @@ const (
|
||||
// CacheConfig contains the configuration values for the trie caching/pruning
|
||||
// that's resident in a blockchain.
|
||||
type CacheConfig struct {
|
||||
TrieCleanLimit int // Memory allowance (MB) to use for caching trie nodes in memory
|
||||
TrieCleanNoPrefetch bool // Whether to disable heuristic state prefetching for followup blocks
|
||||
TrieDirtyLimit int // Memory limit (MB) at which to start flushing dirty trie nodes to disk
|
||||
TrieDirtyDisabled bool // Whether to disable trie write caching and GC altogether (archive node)
|
||||
TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk
|
||||
TrieCleanLimit int // Memory allowance (MB) to use for caching trie nodes in memory
|
||||
TrieCleanNoPrefetch bool // Whether to disable heuristic state prefetching for followup blocks
|
||||
TrieDirtyLimit int // Memory limit (MB) at which to start flushing dirty trie nodes to disk
|
||||
TrieDirtyDisabled bool // Whether to disable trie write caching and GC altogether (archive node)
|
||||
TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk
|
||||
ProcessingStateDiffs bool // Whether statediffs processing should be taken into a account before a trie is pruned
|
||||
}
|
||||
|
||||
// BlockChain represents the canonical chain given a database with a genesis
|
||||
@ -156,6 +157,8 @@ type BlockChain struct {
|
||||
|
||||
badBlocks *lru.Cache // Bad block cache
|
||||
shouldPreserve func(*types.Block) bool // Function used to determine whether should preserve the given block.
|
||||
|
||||
stateDiffsProcessed map[common.Hash]int
|
||||
}
|
||||
|
||||
// NewBlockChain returns a fully initialised block chain using information
|
||||
@ -175,23 +178,24 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
|
||||
blockCache, _ := lru.New(blockCacheLimit)
|
||||
futureBlocks, _ := lru.New(maxFutureBlocks)
|
||||
badBlocks, _ := lru.New(badBlockLimit)
|
||||
|
||||
stateDiffsProcessed := make(map[common.Hash]int)
|
||||
bc := &BlockChain{
|
||||
chainConfig: chainConfig,
|
||||
cacheConfig: cacheConfig,
|
||||
db: db,
|
||||
triegc: prque.New(nil),
|
||||
stateCache: state.NewDatabaseWithCache(db, cacheConfig.TrieCleanLimit),
|
||||
quit: make(chan struct{}),
|
||||
shouldPreserve: shouldPreserve,
|
||||
bodyCache: bodyCache,
|
||||
bodyRLPCache: bodyRLPCache,
|
||||
receiptsCache: receiptsCache,
|
||||
blockCache: blockCache,
|
||||
futureBlocks: futureBlocks,
|
||||
engine: engine,
|
||||
vmConfig: vmConfig,
|
||||
badBlocks: badBlocks,
|
||||
chainConfig: chainConfig,
|
||||
cacheConfig: cacheConfig,
|
||||
db: db,
|
||||
triegc: prque.New(nil),
|
||||
stateCache: state.NewDatabaseWithCache(db, cacheConfig.TrieCleanLimit),
|
||||
quit: make(chan struct{}),
|
||||
shouldPreserve: shouldPreserve,
|
||||
bodyCache: bodyCache,
|
||||
bodyRLPCache: bodyRLPCache,
|
||||
receiptsCache: receiptsCache,
|
||||
blockCache: blockCache,
|
||||
futureBlocks: futureBlocks,
|
||||
engine: engine,
|
||||
vmConfig: vmConfig,
|
||||
badBlocks: badBlocks,
|
||||
stateDiffsProcessed: stateDiffsProcessed,
|
||||
}
|
||||
bc.validator = NewBlockValidator(chainConfig, bc, engine)
|
||||
bc.prefetcher = newStatePrefetcher(chainConfig, bc, engine)
|
||||
@ -932,6 +936,11 @@ func (bc *BlockChain) WriteBlockWithoutState(block *types.Block, td *big.Int) (e
|
||||
return nil
|
||||
}
|
||||
|
||||
func (bc *BlockChain) AddToStateDiffProcessedCollection(hash common.Hash) {
|
||||
count := bc.stateDiffsProcessed[hash]
|
||||
bc.stateDiffsProcessed[hash] = count + 1
|
||||
}
|
||||
|
||||
// WriteBlockWithState writes the block and all associated state to the database.
|
||||
func (bc *BlockChain) WriteBlockWithState(block *types.Block, receipts []*types.Receipt, state *state.StateDB) (status WriteStatus, err error) {
|
||||
bc.chainmu.Lock()
|
||||
@ -1016,6 +1025,16 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
|
||||
bc.triegc.Push(root, number)
|
||||
break
|
||||
}
|
||||
|
||||
if bc.cacheConfig.ProcessingStateDiffs {
|
||||
if !bc.allowedRootToBeDereferenced(root.(common.Hash)) {
|
||||
bc.triegc.Push(root, number)
|
||||
break
|
||||
} else {
|
||||
delete(bc.stateDiffsProcessed, root.(common.Hash))
|
||||
}
|
||||
}
|
||||
|
||||
triedb.Dereference(root.(common.Hash))
|
||||
}
|
||||
}
|
||||
@ -1070,6 +1089,15 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
|
||||
return status, nil
|
||||
}
|
||||
|
||||
// since we need the state tries of the current block and its parent in-memory
|
||||
// in order to process statediffs, we should avoid dereferencing roots until
|
||||
// its statediff and its child have been processed
|
||||
func (bc *BlockChain) allowedRootToBeDereferenced(root common.Hash) bool {
|
||||
diffProcessedForSelfAndChildCount := 2
|
||||
count := bc.stateDiffsProcessed[root]
|
||||
return count >= diffProcessedForSelfAndChildCount
|
||||
}
|
||||
|
||||
// addFutureBlock checks if the block is within the max allowed window to get
|
||||
// accepted for future processing, and returns an error if the block is too far
|
||||
// ahead and was not added.
|
||||
|
@ -1810,3 +1810,84 @@ func TestPrunedImportSide(t *testing.T) {
|
||||
testSideImport(t, 1, 10)
|
||||
testSideImport(t, 1, -10)
|
||||
}
|
||||
|
||||
func TestProcessingStateDiffs(t *testing.T) {
|
||||
defaultTrieCleanCache := 256
|
||||
defaultTrieDirtyCache := 256
|
||||
defaultTrieTimeout := 60 * time.Minute
|
||||
cacheConfig := &CacheConfig{
|
||||
TrieDirtyDisabled: false,
|
||||
TrieCleanLimit: defaultTrieCleanCache,
|
||||
TrieDirtyLimit: defaultTrieDirtyCache,
|
||||
TrieTimeLimit: defaultTrieTimeout,
|
||||
ProcessingStateDiffs: true,
|
||||
}
|
||||
db := rawdb.NewMemoryDatabase()
|
||||
genesis := new(Genesis).MustCommit(db)
|
||||
numberOfBlocks := triesInMemory
|
||||
engine := ethash.NewFaker()
|
||||
blockchain, _ := NewBlockChain(db, cacheConfig, params.AllEthashProtocolChanges, engine, vm.Config{}, nil)
|
||||
blocks := makeBlockChain(genesis, numberOfBlocks+1, engine, db, canonicalSeed)
|
||||
_, err := blockchain.InsertChain(blocks)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create pristine chain: %v", err)
|
||||
}
|
||||
defer blockchain.Stop()
|
||||
|
||||
//when adding a root hash to the collection, it will increment the count
|
||||
firstStateRoot := blocks[0].Root()
|
||||
blockchain.AddToStateDiffProcessedCollection(firstStateRoot)
|
||||
value, ok := blockchain.stateDiffsProcessed[firstStateRoot]
|
||||
if !ok {
|
||||
t.Error("state root not found in collection")
|
||||
}
|
||||
if value != 1 {
|
||||
t.Error("state root count not correct", "want", 1, "got", value)
|
||||
}
|
||||
|
||||
blockchain.AddToStateDiffProcessedCollection(firstStateRoot)
|
||||
value, ok = blockchain.stateDiffsProcessed[firstStateRoot]
|
||||
if !ok {
|
||||
t.Error("state root not found in collection")
|
||||
}
|
||||
if value != 2 {
|
||||
t.Error("state root count not correct", "want", 2, "got", value)
|
||||
}
|
||||
|
||||
moreBlocks := makeBlockChain(blocks[len(blocks)-1], 1, engine, db, canonicalSeed)
|
||||
_, err = blockchain.InsertChain(moreBlocks)
|
||||
|
||||
//a root hash can be dereferenced when it's state diff and it's child's state diff have been processed
|
||||
//(i.e. it has a count of 2 in stateDiffsProcessed)
|
||||
nodes := blockchain.stateCache.TrieDB().Nodes()
|
||||
if containsRootHash(nodes, firstStateRoot) {
|
||||
t.Errorf("stateRoot %s in nodes, want: %t, got: %t", firstStateRoot.Hex(), false, true)
|
||||
}
|
||||
|
||||
//a root hash should still be in the in-mem db if it's child's state diff hasn't yet been processed
|
||||
//(i.e. it has a count of 1 stateDiffsProcessed)
|
||||
secondStateRoot := blocks[1].Root()
|
||||
blockchain.AddToStateDiffProcessedCollection(secondStateRoot)
|
||||
if !containsRootHash(nodes, secondStateRoot) {
|
||||
t.Errorf("stateRoot %s in nodes, want: %t, got: %t", secondStateRoot.Hex(), true, false)
|
||||
}
|
||||
|
||||
//the stateDiffsProcessed collection is cleaned up once a hash has been dereferenced
|
||||
_, ok = blockchain.stateDiffsProcessed[firstStateRoot]
|
||||
if ok {
|
||||
t.Errorf("stateRoot %s in stateDiffsProcessed collection, want: %t, got: %t",
|
||||
firstStateRoot.Hex(),
|
||||
false,
|
||||
ok,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
func containsRootHash(collection []common.Hash, hash common.Hash) bool {
|
||||
for _, n := range collection {
|
||||
if n == hash {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
@ -166,11 +166,12 @@ func New(ctx *node.ServiceContext, config *Config) (*Ethereum, error) {
|
||||
EVMInterpreter: config.EVMInterpreter,
|
||||
}
|
||||
cacheConfig = &core.CacheConfig{
|
||||
TrieCleanLimit: config.TrieCleanCache,
|
||||
TrieCleanNoPrefetch: config.NoPrefetch,
|
||||
TrieDirtyLimit: config.TrieDirtyCache,
|
||||
TrieDirtyDisabled: config.NoPruning,
|
||||
TrieTimeLimit: config.TrieTimeout,
|
||||
TrieCleanLimit: config.TrieCleanCache,
|
||||
TrieCleanNoPrefetch: config.NoPrefetch,
|
||||
TrieDirtyLimit: config.TrieDirtyCache,
|
||||
TrieDirtyDisabled: config.NoPruning,
|
||||
TrieTimeLimit: config.TrieTimeout,
|
||||
ProcessingStateDiffs: config.StateDiff,
|
||||
}
|
||||
)
|
||||
eth.blockchain, err = core.NewBlockChain(chainDb, cacheConfig, chainConfig, eth.engine, vmConfig, eth.shouldPreserve)
|
||||
|
@ -59,6 +59,8 @@ var DefaultConfig = Config{
|
||||
Blocks: 20,
|
||||
Percentile: 60,
|
||||
},
|
||||
|
||||
StateDiff: false,
|
||||
}
|
||||
|
||||
func init() {
|
||||
@ -154,6 +156,8 @@ type Config struct {
|
||||
|
||||
// RPCGasCap is the global gas cap for eth-call variants.
|
||||
RPCGasCap *big.Int `toml:",omitempty"`
|
||||
|
||||
StateDiff bool
|
||||
}
|
||||
|
||||
type configMarshaling struct {
|
||||
|
97
statediff/api.go
Normal file
97
statediff/api.go
Normal file
@ -0,0 +1,97 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package statediff
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
)
|
||||
|
||||
// APIName is the namespace used for the state diffing service API
|
||||
const APIName = "statediff"
|
||||
|
||||
// APIVersion is the version of the state diffing service API
|
||||
const APIVersion = "0.0.1"
|
||||
|
||||
// PublicStateDiffAPI provides the a websocket service
|
||||
// that can be used to stream out state diffs as they
|
||||
// are produced by a full node
|
||||
type PublicStateDiffAPI struct {
|
||||
sds IService
|
||||
|
||||
mu sync.Mutex
|
||||
lastUsed map[string]time.Time // keeps track when a filter was polled for the last time.
|
||||
}
|
||||
|
||||
// NewPublicStateDiffAPI create a new state diff websocket streaming service.
|
||||
func NewPublicStateDiffAPI(sds IService) *PublicStateDiffAPI {
|
||||
return &PublicStateDiffAPI{
|
||||
sds: sds,
|
||||
lastUsed: make(map[string]time.Time),
|
||||
mu: sync.Mutex{},
|
||||
}
|
||||
}
|
||||
|
||||
// Subscribe is the public method to setup a subscription that fires off state-diff payloads as they are created
|
||||
func (api *PublicStateDiffAPI) Subscribe(ctx context.Context) (*rpc.Subscription, error) {
|
||||
|
||||
// ensure that the RPC connection supports subscriptions
|
||||
notifier, supported := rpc.NotifierFromContext(ctx)
|
||||
if !supported {
|
||||
return nil, rpc.ErrNotificationsUnsupported
|
||||
}
|
||||
|
||||
// create subscription and start waiting for statediff events
|
||||
rpcSub := notifier.CreateSubscription()
|
||||
|
||||
go func() {
|
||||
// subscribe to events from the state diff service
|
||||
payloadChannel := make(chan Payload)
|
||||
quitChan := make(chan bool)
|
||||
api.sds.Subscribe(rpcSub.ID, payloadChannel, quitChan)
|
||||
|
||||
// loop and await state diff payloads and relay them to the subscriber with then notifier
|
||||
for {
|
||||
select {
|
||||
case packet := <-payloadChannel:
|
||||
if err := notifier.Notify(rpcSub.ID, packet); err != nil {
|
||||
log.Error("Failed to send state diff packet", "err", err)
|
||||
}
|
||||
case <-rpcSub.Err():
|
||||
err := api.sds.Unsubscribe(rpcSub.ID)
|
||||
if err != nil {
|
||||
log.Error("Failed to unsubscribe from the state diff service", err)
|
||||
}
|
||||
return
|
||||
case <-notifier.Closed():
|
||||
It looks like this It looks like this `Closed()` method is [deprecated](https://github.com/vulcanize/go-ethereum/blob/master/rpc/subscription.go#L142) in favor of an error channel
i-norden
commented
Nice catch! So that means we can just drop this Nice catch! So that means we can just drop this `notifier.Closed()` and the `rpcSub.Err()` should take care of everything?
Yep, that's how I'm interpreting it! Yep, that's how I'm interpreting it!
|
||||
err := api.sds.Unsubscribe(rpcSub.ID)
|
||||
if err != nil {
|
||||
log.Error("Failed to unsubscribe from the state diff service", err)
|
||||
}
|
||||
return
|
||||
case <-quitChan:
|
||||
// don't need to unsubscribe, statediff service does so before sending the quit signal
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return rpcSub, nil
|
||||
}
|
277
statediff/builder.go
Normal file
277
statediff/builder.go
Normal file
@ -0,0 +1,277 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Contains a batch of utility type declarations used by the tests. As the node
|
||||
// operates on unique types, a lot of them are needed to check various features.
|
||||
|
||||
package statediff
|
||||
|
||||
import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
// Builder interface exposes the method for building a state diff between two blocks
|
||||
type Builder interface {
|
||||
BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (StateDiff, error)
|
||||
}
|
||||
|
||||
type builder struct {
|
||||
chainDB ethdb.Database
|
||||
blockChain *core.BlockChain
|
||||
}
|
||||
|
||||
// NewBuilder is used to create a builder
|
||||
func NewBuilder(db ethdb.Database, blockChain *core.BlockChain) Builder {
|
||||
return &builder{
|
||||
chainDB: db,
|
||||
blockChain: blockChain,
|
||||
}
|
||||
}
|
||||
|
||||
// BuildStateDiff builds a StateDiff object from two blocks
|
||||
func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (StateDiff, error) {
|
||||
// Generate tries for old and new states
|
||||
stateCache := sdb.blockChain.StateCache()
|
||||
oldTrie, err := stateCache.OpenTrie(oldStateRoot)
|
||||
if err != nil {
|
||||
log.Error("Error creating trie for oldStateRoot", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
newTrie, err := stateCache.OpenTrie(newStateRoot)
|
||||
if err != nil {
|
||||
log.Error("Error creating trie for newStateRoot", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
|
||||
// Find created accounts
|
||||
oldIt := oldTrie.NodeIterator([]byte{})
|
||||
newIt := newTrie.NodeIterator([]byte{})
|
||||
creations, err := sdb.collectDiffNodes(oldIt, newIt)
|
||||
if err != nil {
|
||||
log.Error("Error collecting creation diff nodes", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
|
||||
// Find deleted accounts
|
||||
oldIt = oldTrie.NodeIterator([]byte{})
|
||||
newIt = newTrie.NodeIterator([]byte{})
|
||||
deletions, err := sdb.collectDiffNodes(newIt, oldIt)
|
||||
if err != nil {
|
||||
log.Error("Error collecting deletion diff nodes", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
|
||||
// Find all the diffed keys
|
||||
createKeys := sortKeys(creations)
|
||||
deleteKeys := sortKeys(deletions)
|
||||
updatedKeys := findIntersection(createKeys, deleteKeys)
|
||||
|
||||
// Build and return the statediff
|
||||
updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, updatedKeys)
|
||||
if err != nil {
|
||||
log.Error("Error building diff for updated accounts", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
createdAccounts, err := sdb.buildDiffEventual(creations)
|
||||
if err != nil {
|
||||
log.Error("Error building diff for created accounts", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
deletedAccounts, err := sdb.buildDiffEventual(deletions)
|
||||
if err != nil {
|
||||
log.Error("Error building diff for deleted accounts", "error", err)
|
||||
return StateDiff{}, err
|
||||
}
|
||||
|
||||
return StateDiff{
|
||||
BlockNumber: blockNumber,
|
||||
BlockHash: blockHash,
|
||||
CreatedAccounts: createdAccounts,
|
||||
DeletedAccounts: deletedAccounts,
|
||||
UpdatedAccounts: updatedAccounts,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (AccountsMap, error) {
|
||||
var diffAccounts = make(AccountsMap)
|
||||
it, _ := trie.NewDifferenceIterator(a, b)
|
||||
|
||||
for {
|
||||
log.Debug("Current Path and Hash", "path", pathToStr(it), "hashold", it.Hash())
|
||||
if it.Leaf() {
|
||||
leafProof := make([][]byte, len(it.LeafProof()))
|
||||
copy(leafProof, it.LeafProof())
|
||||
leafPath := make([]byte, len(it.Path()))
|
||||
copy(leafPath, it.Path())
|
||||
leafKey := make([]byte, len(it.LeafKey()))
|
||||
copy(leafKey, it.LeafKey())
|
||||
leafKeyHash := common.BytesToHash(leafKey)
|
||||
leafValue := make([]byte, len(it.LeafBlob()))
|
||||
copy(leafValue, it.LeafBlob())
|
||||
// lookup account state
|
||||
var account state.Account
|
||||
if err := rlp.DecodeBytes(leafValue, &account); err != nil {
|
||||
log.Error("Error looking up account via address", "address", leafKeyHash, "error", err)
|
||||
return nil, err
|
||||
}
|
||||
aw := accountWrapper{
|
||||
Account: account,
|
||||
RawKey: leafKey,
|
||||
RawValue: leafValue,
|
||||
Proof: leafProof,
|
||||
Path: leafPath,
|
||||
}
|
||||
// record account to diffs (creation if we are looking at new - old; deletion if old - new)
|
||||
log.Debug("Account lookup successful", "address", leafKeyHash, "account", account)
|
||||
diffAccounts[leafKeyHash] = aw
|
||||
}
|
||||
cont := it.Next(true)
|
||||
if !cont {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return diffAccounts, nil
|
||||
}
|
||||
|
||||
func (sdb *builder) buildDiffEventual(accounts AccountsMap) (AccountDiffsMap, error) {
|
||||
accountDiffs := make(AccountDiffsMap)
|
||||
for _, val := range accounts {
|
||||
storageDiffs, err := sdb.buildStorageDiffsEventual(val.Account.Root)
|
||||
if err != nil {
|
||||
log.Error("Failed building eventual storage diffs", "Address", common.BytesToHash(val.RawKey), "error", err)
|
||||
return nil, err
|
||||
}
|
||||
accountDiffs[common.BytesToHash(val.RawKey)] = AccountDiff{
|
||||
Key: val.RawKey,
|
||||
Value: val.RawValue,
|
||||
Proof: val.Proof,
|
||||
Path: val.Path,
|
||||
Storage: storageDiffs,
|
||||
}
|
||||
}
|
||||
|
||||
return accountDiffs, nil
|
||||
}
|
||||
|
||||
func (sdb *builder) buildDiffIncremental(creations AccountsMap, deletions AccountsMap, updatedKeys []string) (AccountDiffsMap, error) {
|
||||
updatedAccounts := make(AccountDiffsMap)
|
||||
for _, val := range updatedKeys {
|
||||
createdAcc := creations[common.HexToHash(val)]
|
||||
deletedAcc := deletions[common.HexToHash(val)]
|
||||
oldSR := deletedAcc.Account.Root
|
||||
newSR := createdAcc.Account.Root
|
||||
storageDiffs, err := sdb.buildStorageDiffsIncremental(oldSR, newSR)
|
||||
if err != nil {
|
||||
log.Error("Failed building storage diffs", "Address", val, "error", err)
|
||||
return nil, err
|
||||
}
|
||||
updatedAccounts[common.HexToHash(val)] = AccountDiff{
|
||||
Key: createdAcc.RawKey,
|
||||
Value: createdAcc.RawValue,
|
||||
Proof: createdAcc.Proof,
|
||||
Path: createdAcc.Path,
|
||||
Storage: storageDiffs,
|
||||
}
|
||||
delete(creations, common.HexToHash(val))
|
||||
delete(deletions, common.HexToHash(val))
|
||||
}
|
||||
|
||||
return updatedAccounts, nil
|
||||
}
|
||||
|
||||
func (sdb *builder) buildStorageDiffsEventual(sr common.Hash) ([]StorageDiff, error) {
|
||||
log.Debug("Storage Root For Eventual Diff", "root", sr.Hex())
|
||||
stateCache := sdb.blockChain.StateCache()
|
||||
sTrie, err := stateCache.OpenTrie(sr)
|
||||
if err != nil {
|
||||
log.Info("error in build storage diff eventual", "error", err)
|
||||
return nil, err
|
||||
}
|
||||
it := sTrie.NodeIterator(make([]byte, 0))
|
||||
storageDiffs := buildStorageDiffsFromTrie(it)
|
||||
return storageDiffs, nil
|
||||
}
|
||||
|
||||
func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) ([]StorageDiff, error) {
|
||||
log.Debug("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex())
|
||||
stateCache := sdb.blockChain.StateCache()
|
||||
|
||||
oldTrie, err := stateCache.OpenTrie(oldSR)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
newTrie, err := stateCache.OpenTrie(newSR)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
oldIt := oldTrie.NodeIterator(make([]byte, 0))
|
||||
newIt := newTrie.NodeIterator(make([]byte, 0))
|
||||
it, _ := trie.NewDifferenceIterator(oldIt, newIt)
|
||||
storageDiffs := buildStorageDiffsFromTrie(it)
|
||||
|
||||
return storageDiffs, nil
|
||||
}
|
||||
|
||||
func buildStorageDiffsFromTrie(it trie.NodeIterator) []StorageDiff {
|
||||
storageDiffs := make([]StorageDiff, 0)
|
||||
for {
|
||||
log.Debug("Iterating over state at path ", "path", pathToStr(it))
|
||||
if it.Leaf() {
|
||||
log.Debug("Found leaf in storage", "path", pathToStr(it))
|
||||
leafProof := make([][]byte, len(it.LeafProof()))
|
||||
copy(leafProof, it.LeafProof())
|
||||
leafPath := make([]byte, len(it.Path()))
|
||||
copy(leafPath, it.Path())
|
||||
leafKey := make([]byte, len(it.LeafKey()))
|
||||
copy(leafKey, it.LeafKey())
|
||||
leafValue := make([]byte, len(it.LeafBlob()))
|
||||
copy(leafValue, it.LeafBlob())
|
||||
storageDiffs = append(storageDiffs, StorageDiff{
|
||||
Key: leafKey,
|
||||
Value: leafValue,
|
||||
Path: leafPath,
|
||||
Proof: leafProof,
|
||||
})
|
||||
}
|
||||
cont := it.Next(true)
|
||||
if !cont {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return storageDiffs
|
||||
}
|
||||
|
||||
func (sdb *builder) addressByPath(path []byte) (*common.Address, error) {
|
||||
log.Debug("Looking up address from path", "path", hexutil.Encode(append([]byte("secure-key-"), path...)))
|
||||
addrBytes, err := sdb.chainDB.Get(append([]byte("secure-key-"), hexToKeyBytes(path)...))
|
||||
if err != nil {
|
||||
log.Error("Error looking up address via path", "path", hexutil.Encode(append([]byte("secure-key-"), path...)), "error", err)
|
||||
return nil, err
|
||||
}
|
||||
addr := common.BytesToAddress(addrBytes)
|
||||
log.Debug("Address found", "Address", addr)
|
||||
return &addr, nil
|
||||
}
|
392
statediff/builder_test.go
Normal file
392
statediff/builder_test.go
Normal file
@ -0,0 +1,392 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package statediff_test
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/big"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
"github.com/ethereum/go-ethereum/statediff/testhelpers"
|
||||
)
|
||||
|
||||
var (
|
||||
contractLeafKey common.Hash
|
||||
emptyAccountDiffEventualMap = make(statediff.AccountDiffsMap)
|
||||
emptyAccountDiffIncrementalMap = make(statediff.AccountDiffsMap)
|
||||
block0Hash, block1Hash, block2Hash, block3Hash common.Hash
|
||||
block0, block1, block2, block3 *types.Block
|
||||
builder statediff.Builder
|
||||
miningReward = int64(2000000000000000000)
|
||||
burnAddress = common.HexToAddress("0x0")
|
||||
burnLeafKey = testhelpers.AddressToLeafKey(burnAddress)
|
||||
)
|
||||
|
||||
func TestBuilder(t *testing.T) {
|
||||
_, blockMap, chain := testhelpers.MakeChain(3, testhelpers.Genesis)
|
||||
contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr)
|
||||
defer chain.Stop()
|
||||
block0Hash = common.HexToHash("0xd1721cfd0b29c36fd7a68f25c128e86413fb666a6e1d68e89b875bd299262661")
|
||||
block1Hash = common.HexToHash("0xbbe88de60ba33a3f18c0caa37d827bfb70252e19e40a07cd34041696c35ecb1a")
|
||||
block2Hash = common.HexToHash("0x34ad0fd9bb2911986b75d518c822641079dea823bc6952343ebf05da1062b6f5")
|
||||
block3Hash = common.HexToHash("0x9872058136c560a6ebed0c0522b8d3016fc21f4fb0fb6585ddd8fd4c54f9909a")
|
||||
|
||||
block0 = blockMap[block0Hash]
|
||||
block1 = blockMap[block1Hash]
|
||||
block2 = blockMap[block2Hash]
|
||||
block3 = blockMap[block3Hash]
|
||||
builder = statediff.NewBuilder(testhelpers.Testdb, chain)
|
||||
|
||||
type arguments struct {
|
||||
oldStateRoot common.Hash
|
||||
newStateRoot common.Hash
|
||||
blockNumber int64
|
||||
blockHash common.Hash
|
||||
}
|
||||
|
||||
var (
|
||||
balanceChange10000 = int64(10000)
|
||||
balanceChange1000 = int64(1000)
|
||||
block1BankBalance = int64(99990000)
|
||||
block1Account1Balance = int64(10000)
|
||||
block2Account2Balance = int64(1000)
|
||||
nonce0 = uint64(0)
|
||||
nonce1 = uint64(1)
|
||||
nonce2 = uint64(2)
|
||||
nonce3 = uint64(3)
|
||||
originalContractRoot = "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"
|
||||
contractContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0"
|
||||
newContractRoot = "0x71e0d14b2b93e5c7f9748e69e1fe5f17498a1c3ac3cec29f96af13d7f8a4e070"
|
||||
originalStorageLocation = common.HexToHash("0")
|
||||
originalStorageKey = crypto.Keccak256Hash(originalStorageLocation[:]).Bytes()
|
||||
updatedStorageLocation = common.HexToHash("2")
|
||||
updatedStorageKey = crypto.Keccak256Hash(updatedStorageLocation[:]).Bytes()
|
||||
originalStorageValue = common.Hex2Bytes("01")
|
||||
updatedStorageValue = common.Hex2Bytes("03")
|
||||
account1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce0,
|
||||
Balance: big.NewInt(balanceChange10000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
burnAccount1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce0,
|
||||
Balance: big.NewInt(miningReward),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
bankAccount1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce1,
|
||||
Balance: big.NewInt(testhelpers.TestBankFunds.Int64() - balanceChange10000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
account2, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce0,
|
||||
Balance: big.NewInt(balanceChange1000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
contractAccount, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce1,
|
||||
Balance: big.NewInt(0),
|
||||
CodeHash: common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea").Bytes(),
|
||||
Root: common.HexToHash(contractContractRoot),
|
||||
})
|
||||
bankAccount2, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce2,
|
||||
Balance: big.NewInt(block1BankBalance - balanceChange1000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
account3, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce2,
|
||||
Balance: big.NewInt(block1Account1Balance - balanceChange1000 + balanceChange1000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
burnAccount2, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce0,
|
||||
Balance: big.NewInt(miningReward + miningReward),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
account4, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce0,
|
||||
Balance: big.NewInt(block2Account2Balance + miningReward),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
contractAccount2, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce1,
|
||||
Balance: big.NewInt(0),
|
||||
CodeHash: common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea").Bytes(),
|
||||
Root: common.HexToHash(newContractRoot),
|
||||
})
|
||||
bankAccount3, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: nonce3,
|
||||
Balance: big.NewInt(99989000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash(originalContractRoot),
|
||||
})
|
||||
)
|
||||
|
||||
var tests = []struct {
|
||||
name string
|
||||
startingArguments arguments
|
||||
expected *statediff.StateDiff
|
||||
}{
|
||||
{
|
||||
"testEmptyDiff",
|
||||
arguments{
|
||||
oldStateRoot: block0.Root(),
|
||||
newStateRoot: block0.Root(),
|
||||
blockNumber: block0.Number().Int64(),
|
||||
blockHash: block0Hash,
|
||||
},
|
||||
&statediff.StateDiff{
|
||||
BlockNumber: block0.Number().Int64(),
|
||||
BlockHash: block0Hash,
|
||||
CreatedAccounts: emptyAccountDiffEventualMap,
|
||||
DeletedAccounts: emptyAccountDiffEventualMap,
|
||||
UpdatedAccounts: emptyAccountDiffIncrementalMap,
|
||||
},
|
||||
},
|
||||
{
|
||||
"testBlock1",
|
||||
//10000 transferred from testBankAddress to account1Addr
|
||||
arguments{
|
||||
oldStateRoot: block0.Root(),
|
||||
newStateRoot: block1.Root(),
|
||||
blockNumber: block1.Number().Int64(),
|
||||
blockHash: block1Hash,
|
||||
},
|
||||
&statediff.StateDiff{
|
||||
BlockNumber: block1.Number().Int64(),
|
||||
BlockHash: block1.Hash(),
|
||||
CreatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.Account1LeafKey: {
|
||||
Key: testhelpers.Account1LeafKey.Bytes(),
|
||||
Value: account1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 128, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
burnLeafKey: {
|
||||
Key: burnLeafKey.Bytes(),
|
||||
Value: burnAccount1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
DeletedAccounts: emptyAccountDiffEventualMap,
|
||||
UpdatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.BankLeafKey: {
|
||||
Key: testhelpers.BankLeafKey.Bytes(),
|
||||
Value: bankAccount1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 1, 132, 5, 245, 185, 240, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
"testBlock2",
|
||||
//1000 transferred from testBankAddress to account1Addr
|
||||
//1000 transferred from account1Addr to account2Addr
|
||||
arguments{
|
||||
oldStateRoot: block1.Root(),
|
||||
newStateRoot: block2.Root(),
|
||||
blockNumber: block2.Number().Int64(),
|
||||
blockHash: block2Hash,
|
||||
},
|
||||
&statediff.StateDiff{
|
||||
BlockNumber: block2.Number().Int64(),
|
||||
BlockHash: block2.Hash(),
|
||||
CreatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.Account2LeafKey: {
|
||||
Key: testhelpers.Account2LeafKey.Bytes(),
|
||||
Value: account2,
|
||||
Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 107, 160, 57, 87, 243, 226, 240, 74, 7, 100, 195, 160, 73, 27, 23, 95, 105, 146, 109, 166, 30, 251, 204, 143, 97, 250, 20, 85, 253, 45, 43, 76, 221, 69, 184, 72, 248, 70, 128, 130, 3, 232, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{12, 9, 5, 7, 15, 3, 14, 2, 15, 0, 4, 10, 0, 7, 6, 4, 12, 3, 10, 0, 4, 9, 1, 11, 1, 7, 5, 15, 6, 9, 9, 2, 6, 13, 10, 6, 1, 14, 15, 11, 12, 12, 8, 15, 6, 1, 15, 10, 1, 4, 5, 5, 15, 13, 2, 13, 2, 11, 4, 12, 13, 13, 4, 5, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
contractLeafKey: {
|
||||
Key: contractLeafKey.Bytes(),
|
||||
Value: contractAccount,
|
||||
Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 130, 30, 37, 86, 162, 144, 200, 100, 5, 248, 22, 10, 45, 102, 32, 66, 164, 49, 186, 69, 107, 157, 178, 101, 199, 155, 184, 55, 192, 75, 229, 240, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}},
|
||||
Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16},
|
||||
Storage: []statediff.StorageDiff{
|
||||
{
|
||||
Key: originalStorageKey,
|
||||
Value: originalStorageValue,
|
||||
Proof: [][]byte{{227, 161, 32, 41, 13, 236, 217, 84, 139, 98, 168, 214, 3, 69, 169, 136, 56, 111, 200, 75, 166, 188, 149, 72, 64, 8, 246, 54, 47, 147, 22, 14, 243, 229, 99, 1}},
|
||||
Path: []byte{2, 9, 0, 13, 14, 12, 13, 9, 5, 4, 8, 11, 6, 2, 10, 8, 13, 6, 0, 3, 4, 5, 10, 9, 8, 8, 3, 8, 6, 15, 12, 8, 4, 11, 10, 6, 11, 12, 9, 5, 4, 8, 4, 0, 0, 8, 15, 6, 3, 6, 2, 15, 9, 3, 1, 6, 0, 14, 15, 3, 14, 5, 6, 3, 16},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
DeletedAccounts: emptyAccountDiffEventualMap,
|
||||
UpdatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.BankLeafKey: {
|
||||
Key: testhelpers.BankLeafKey.Bytes(),
|
||||
Value: bankAccount2,
|
||||
Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 2, 132, 5, 245, 182, 8, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
testhelpers.Account1LeafKey: {
|
||||
Key: testhelpers.Account1LeafKey.Bytes(),
|
||||
Value: account3,
|
||||
Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 2, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
burnLeafKey: {
|
||||
Key: burnLeafKey.Bytes(),
|
||||
Value: burnAccount2,
|
||||
Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 55, 130, 218, 206, 157, 144, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
"testBlock3",
|
||||
//the contract's storage is changed
|
||||
//and the block is mined by account 2
|
||||
arguments{
|
||||
oldStateRoot: block2.Root(),
|
||||
newStateRoot: block3.Root(),
|
||||
blockNumber: block3.Number().Int64(),
|
||||
blockHash: block3.Hash(),
|
||||
},
|
||||
&statediff.StateDiff{
|
||||
BlockNumber: block3.Number().Int64(),
|
||||
BlockHash: block3.Hash(),
|
||||
CreatedAccounts: statediff.AccountDiffsMap{},
|
||||
DeletedAccounts: emptyAccountDiffEventualMap,
|
||||
UpdatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.Account2LeafKey: {
|
||||
Key: testhelpers.Account2LeafKey.Bytes(),
|
||||
Value: account4,
|
||||
Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 113, 160, 57, 87, 243, 226, 240, 74, 7, 100, 195, 160, 73, 27, 23, 95, 105, 146, 109, 166, 30, 251, 204, 143, 97, 250, 20, 85, 253, 45, 43, 76, 221, 69, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 3, 232, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{12, 9, 5, 7, 15, 3, 14, 2, 15, 0, 4, 10, 0, 7, 6, 4, 12, 3, 10, 0, 4, 9, 1, 11, 1, 7, 5, 15, 6, 9, 9, 2, 6, 13, 10, 6, 1, 14, 15, 11, 12, 12, 8, 15, 6, 1, 15, 10, 1, 4, 5, 5, 15, 13, 2, 13, 2, 11, 4, 12, 13, 13, 4, 5, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
contractLeafKey: {
|
||||
Key: contractLeafKey.Bytes(),
|
||||
Value: contractAccount2,
|
||||
Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 113, 224, 209, 75, 43, 147, 229, 199, 249, 116, 142, 105, 225, 254, 95, 23, 73, 138, 28, 58, 195, 206, 194, 159, 150, 175, 19, 215, 248, 164, 224, 112, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}},
|
||||
Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16},
|
||||
Storage: []statediff.StorageDiff{
|
||||
{
|
||||
Key: updatedStorageKey,
|
||||
Value: updatedStorageValue,
|
||||
Proof: [][]byte{{248, 81, 128, 128, 160, 79, 197, 241, 58, 178, 249, 186, 12, 45, 168, 139, 1, 81, 171, 14, 124, 244, 216, 93, 8, 204, 164, 92, 205, 146, 60, 106, 183, 99, 35, 235, 40, 128, 160, 205, 69, 114, 89, 105, 97, 21, 35, 94, 100, 199, 130, 35, 52, 214, 33, 41, 226, 241, 96, 68, 37, 167, 218, 100, 148, 243, 95, 196, 91, 229, 24, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128},
|
||||
{226, 160, 48, 87, 135, 250, 18, 168, 35, 224, 242, 183, 99, 28, 196, 27, 59, 168, 130, 139, 51, 33, 202, 129, 17, 17, 250, 117, 205, 58, 163, 187, 90, 206, 3}},
|
||||
Path: []byte{4, 0, 5, 7, 8, 7, 15, 10, 1, 2, 10, 8, 2, 3, 14, 0, 15, 2, 11, 7, 6, 3, 1, 12, 12, 4, 1, 11, 3, 11, 10, 8, 8, 2, 8, 11, 3, 3, 2, 1, 12, 10, 8, 1, 1, 1, 1, 1, 15, 10, 7, 5, 12, 13, 3, 10, 10, 3, 11, 11, 5, 10, 12, 14, 16},
|
||||
},
|
||||
},
|
||||
},
|
||||
testhelpers.BankLeafKey: {
|
||||
Key: testhelpers.BankLeafKey.Bytes(),
|
||||
Value: bankAccount3,
|
||||
Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128},
|
||||
{248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 3, 132, 5, 245, 182, 8, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
arguments := test.startingArguments
|
||||
diff, err := builder.BuildStateDiff(arguments.oldStateRoot, arguments.newStateRoot, arguments.blockNumber, arguments.blockHash)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
fields := []string{"BlockNumber", "BlockHash", "DeletedAccounts", "UpdatedAccounts", "CreatedAccounts"}
|
||||
|
||||
for _, field := range fields {
|
||||
reflectionOfDiff := reflect.ValueOf(diff)
|
||||
diffValue := reflect.Indirect(reflectionOfDiff).FieldByName(field)
|
||||
|
||||
reflectionOfExpected := reflect.ValueOf(test.expected)
|
||||
expectedValue := reflect.Indirect(reflectionOfExpected).FieldByName(field)
|
||||
|
||||
diffValueInterface := diffValue.Interface()
|
||||
expectedValueInterface := expectedValue.Interface()
|
||||
|
||||
if !equals(diffValueInterface, expectedValueInterface) {
|
||||
t.Logf("Test failed: %s", test.name)
|
||||
t.Errorf("field: %+v\nactual: %+v\nexpected: %+v", field, diffValueInterface, expectedValueInterface)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func equals(actual, expected interface{}) (success bool) {
|
||||
if actualByteSlice, ok := actual.([]byte); ok {
|
||||
if expectedByteSlice, ok := expected.([]byte); ok {
|
||||
return bytes.Equal(actualByteSlice, expectedByteSlice)
|
||||
}
|
||||
}
|
||||
|
||||
return reflect.DeepEqual(actual, expected)
|
||||
}
|
||||
|
||||
/*
|
||||
contract test {
|
||||
|
||||
uint256[100] data;
|
||||
|
||||
constructor() public {
|
||||
data = [1];
|
||||
}
|
||||
|
||||
function Put(uint256 addr, uint256 value) {
|
||||
data[addr] = value;
|
||||
}
|
||||
|
||||
function Get(uint256 addr) constant returns (uint256 value) {
|
||||
return data[addr];
|
||||
}
|
||||
}
|
||||
*/
|
118
statediff/helpers.go
Normal file
118
statediff/helpers.go
Normal file
@ -0,0 +1,118 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Contains a batch of utility type declarations used by the tests. As the node
|
||||
// operates on unique types, a lot of them are needed to check various features.
|
||||
|
||||
package statediff
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"strings"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
func sortKeys(data AccountsMap) []string {
|
||||
var keys []string
|
||||
for key := range data {
|
||||
keys = append(keys, key.Hex())
|
||||
}
|
||||
sort.Strings(keys)
|
||||
|
||||
return keys
|
||||
}
|
||||
|
||||
// BytesToNiblePath
|
||||
func bytesToNiblePath(path []byte) string {
|
||||
if hasTerm(path) {
|
||||
path = path[:len(path)-1]
|
||||
}
|
||||
nibblePath := ""
|
||||
for i, v := range common.ToHex(path) {
|
||||
if i%2 == 0 && i > 1 {
|
||||
continue
|
||||
}
|
||||
nibblePath = nibblePath + string(v)
|
||||
}
|
||||
|
||||
return nibblePath
|
||||
}
|
||||
|
||||
func findIntersection(a, b []string) []string {
|
||||
lenA := len(a)
|
||||
lenB := len(b)
|
||||
iOfA, iOfB := 0, 0
|
||||
updates := make([]string, 0)
|
||||
if iOfA >= lenA || iOfB >= lenB {
|
||||
return updates
|
||||
}
|
||||
for {
|
||||
switch strings.Compare(a[iOfA], b[iOfB]) {
|
||||
// a[iOfA] < b[iOfB]
|
||||
case -1:
|
||||
iOfA++
|
||||
if iOfA >= lenA {
|
||||
return updates
|
||||
}
|
||||
// a[iOfA] == b[iOfB]
|
||||
case 0:
|
||||
updates = append(updates, a[iOfA])
|
||||
iOfA++
|
||||
iOfB++
|
||||
if iOfA >= lenA || iOfB >= lenB {
|
||||
return updates
|
||||
}
|
||||
// a[iOfA] > b[iOfB]
|
||||
case 1:
|
||||
iOfB++
|
||||
if iOfB >= lenB {
|
||||
return updates
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func pathToStr(it trie.NodeIterator) string {
|
||||
return bytesToNiblePath(it.Path())
|
||||
}
|
||||
|
||||
// Duplicated from trie/encoding.go
|
||||
func hexToKeyBytes(hex []byte) []byte {
|
||||
if hasTerm(hex) {
|
||||
hex = hex[:len(hex)-1]
|
||||
}
|
||||
if len(hex)&1 != 0 {
|
||||
panic("can't convert hex key of odd length")
|
||||
}
|
||||
key := make([]byte, (len(hex)+1)/2)
|
||||
decodeNibbles(hex, key)
|
||||
|
||||
return key
|
||||
}
|
||||
|
||||
func decodeNibbles(nibbles []byte, bytes []byte) {
|
||||
for bi, ni := 0, 0; ni < len(nibbles); bi, ni = bi+1, ni+2 {
|
||||
bytes[bi] = nibbles[ni]<<4 | nibbles[ni+1]
|
||||
}
|
||||
}
|
||||
|
||||
// hasTerm returns whether a hex key has the terminator flag.
|
||||
func hasTerm(s []byte) bool {
|
||||
return len(s) > 0 && s[len(s)-1] == 16
|
||||
}
|
244
statediff/service.go
Normal file
244
statediff/service.go
Normal file
@ -0,0 +1,244 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package statediff
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/node"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/event"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/p2p"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
)
|
||||
|
||||
type blockChain interface {
|
||||
SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription
|
||||
GetBlockByHash(hash common.Hash) *types.Block
|
||||
AddToStateDiffProcessedCollection(hash common.Hash)
|
||||
}
|
||||
|
||||
// IService is the state-diffing service interface
|
||||
type IService interface {
|
||||
// APIs(), Protocols(), Start() and Stop()
|
||||
node.Service
|
||||
// Main event loop for processing state diffs
|
||||
Loop(chainEventCh chan core.ChainEvent)
|
||||
// Method to subscribe to receive state diff processing output
|
||||
Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool)
|
||||
// Method to unsubscribe from state diff processing
|
||||
Unsubscribe(id rpc.ID) error
|
||||
}
|
||||
|
||||
// Service is the underlying struct for the state diffing service
|
||||
type Service struct {
|
||||
// Used to sync access to the Subscriptions
|
||||
sync.Mutex
|
||||
// Used to build the state diff objects
|
||||
Builder Builder
|
||||
// Used to subscribe to chain events (blocks)
|
||||
BlockChain blockChain
|
||||
// Used to signal shutdown of the service
|
||||
QuitChan chan bool
|
||||
// A mapping of rpc.IDs to their subscription channels
|
||||
Subscriptions map[rpc.ID]Subscription
|
||||
}
|
||||
|
||||
// Subscription struct holds our subscription channels
|
||||
type Subscription struct {
|
||||
PayloadChan chan<- Payload
|
||||
QuitChan chan<- bool
|
||||
}
|
||||
|
||||
// Payload packages the data to send to StateDiffingService subscriptions
|
||||
type Payload struct {
|
||||
BlockRlp []byte `json:"blockRlp" gencodec:"required"`
|
||||
StateDiffRlp []byte `json:"stateDiffRlp" gencodec:"required"`
|
||||
Err error `json:"error"`
|
||||
}
|
||||
|
||||
// NewStateDiffService creates a new StateDiffingService
|
||||
func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*Service, error) {
|
||||
return &Service{
|
||||
Mutex: sync.Mutex{},
|
||||
BlockChain: blockChain,
|
||||
Builder: NewBuilder(db, blockChain),
|
||||
QuitChan: make(chan bool),
|
||||
Subscriptions: make(map[rpc.ID]Subscription),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Protocols exports the services p2p protocols, this service has none
|
||||
func (sds *Service) Protocols() []p2p.Protocol {
|
||||
return []p2p.Protocol{}
|
||||
}
|
||||
|
||||
// APIs returns the RPC descriptors the StateDiffingService offers
|
||||
func (sds *Service) APIs() []rpc.API {
|
||||
return []rpc.API{
|
||||
{
|
||||
Namespace: APIName,
|
||||
Version: APIVersion,
|
||||
Service: NewPublicStateDiffAPI(sds),
|
||||
Public: true,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Loop is the main processing method
|
||||
func (sds *Service) Loop(chainEventCh chan core.ChainEvent) {
|
||||
|
||||
chainEventSub := sds.BlockChain.SubscribeChainEvent(chainEventCh)
|
||||
defer chainEventSub.Unsubscribe()
|
||||
|
||||
blocksCh := make(chan *types.Block, 10)
|
||||
errCh := chainEventSub.Err()
|
||||
|
||||
go func() {
|
||||
HandleChainEventChLoop:
|
||||
for {
|
||||
select {
|
||||
//Notify chain event channel of events
|
||||
case chainEvent := <-chainEventCh:
|
||||
log.Debug("Event received from chainEventCh", "event", chainEvent)
|
||||
blocksCh <- chainEvent.Block
|
||||
//if node stopped
|
||||
case err := <-errCh:
|
||||
log.Warn("Error from chain event subscription, breaking loop.", "error", err)
|
||||
close(sds.QuitChan)
|
||||
break HandleChainEventChLoop
|
||||
case <-sds.QuitChan:
|
||||
break HandleChainEventChLoop
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
//loop through chain events until no more
|
||||
HandleBlockChLoop:
|
||||
for {
|
||||
select {
|
||||
case block := <-blocksCh:
|
||||
currentBlock := block
|
||||
parentHash := currentBlock.ParentHash()
|
||||
parentBlock := sds.BlockChain.GetBlockByHash(parentHash)
|
||||
if parentBlock == nil {
|
||||
log.Error("Parent block is nil, skipping this block",
|
||||
"parent block hash", parentHash.String(),
|
||||
"current block number", currentBlock.Number())
|
||||
break HandleBlockChLoop
|
||||
}
|
||||
|
||||
stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number().Int64(), currentBlock.Hash())
|
||||
if err != nil {
|
||||
log.Error("Error building statediff", "block number", currentBlock.Number(), "error", err)
|
||||
}
|
||||
rlpBuff := new(bytes.Buffer)
|
||||
currentBlock.EncodeRLP(rlpBuff)
|
||||
blockRlp := rlpBuff.Bytes()
|
||||
stateDiffRlp, _ := rlp.EncodeToBytes(stateDiff)
|
||||
payload := Payload{
|
||||
BlockRlp: blockRlp,
|
||||
StateDiffRlp: stateDiffRlp,
|
||||
Err: err,
|
||||
}
|
||||
// If we have any websocket subscription listening in, send the data to them
|
||||
sds.send(payload)
|
||||
case <-sds.QuitChan:
|
||||
log.Debug("Quitting the statediff block channel")
|
||||
sds.close()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Subscribe is used by the API to subscribe to the StateDiffingService loop
|
||||
func (sds *Service) Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool) {
|
||||
log.Info("Subscribing to the statediff service")
|
||||
sds.Lock()
|
||||
sds.Subscriptions[id] = Subscription{
|
||||
PayloadChan: sub,
|
||||
QuitChan: quitChan,
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
||||
|
||||
// Unsubscribe is used to unsubscribe to the StateDiffingService loop
|
||||
func (sds *Service) Unsubscribe(id rpc.ID) error {
|
||||
log.Info("Unsubscribing from the statediff service")
|
||||
sds.Lock()
|
||||
_, ok := sds.Subscriptions[id]
|
||||
if !ok {
|
||||
return fmt.Errorf("cannot unsubscribe; subscription for id %s does not exist", id)
|
||||
}
|
||||
delete(sds.Subscriptions, id)
|
||||
sds.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Start is used to begin the StateDiffingService
|
||||
func (sds *Service) Start(*p2p.Server) error {
|
||||
log.Info("Starting statediff service")
|
||||
|
||||
chainEventCh := make(chan core.ChainEvent, 10)
|
||||
go sds.Loop(chainEventCh)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop is used to close down the StateDiffingService
|
||||
func (sds *Service) Stop() error {
|
||||
log.Info("Stopping statediff service")
|
||||
close(sds.QuitChan)
|
||||
return nil
|
||||
}
|
||||
|
||||
// send is used to fan out and serve a payload to any subscriptions
|
||||
func (sds *Service) send(payload Payload) {
|
||||
sds.Lock()
|
||||
for id, sub := range sds.Subscriptions {
|
||||
select {
|
||||
case sub.PayloadChan <- payload:
|
||||
log.Info("sending state diff payload to subscription %s", id)
|
||||
default:
|
||||
log.Info("unable to send payload to subscription %s; channel has no receiver", id)
|
||||
}
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
||||
|
||||
// close is used to close all listening subscriptions
|
||||
func (sds *Service) close() {
|
||||
sds.Lock()
|
||||
for id, sub := range sds.Subscriptions {
|
||||
select {
|
||||
case sub.QuitChan <- true:
|
||||
delete(sds.Subscriptions, id)
|
||||
log.Info("closing subscription %s", id)
|
||||
default:
|
||||
log.Info("unable to close subscription %s; channel has no receiver", id)
|
||||
}
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
130
statediff/service_test.go
Normal file
130
statediff/service_test.go
Normal file
@ -0,0 +1,130 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package statediff_test
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/big"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
"github.com/ethereum/go-ethereum/statediff/testhelpers/mocks"
|
||||
)
|
||||
|
||||
func TestServiceLoop(t *testing.T) {
|
||||
testErrorInChainEventLoop(t)
|
||||
testErrorInBlockLoop(t)
|
||||
}
|
||||
|
||||
var (
|
||||
eventsChannel = make(chan core.ChainEvent, 1)
|
||||
|
||||
parentRoot1 = common.HexToHash("0x01")
|
||||
parentRoot2 = common.HexToHash("0x02")
|
||||
parentHeader1 = types.Header{Number: big.NewInt(rand.Int63()), Root: parentRoot1}
|
||||
parentHeader2 = types.Header{Number: big.NewInt(rand.Int63()), Root: parentRoot2}
|
||||
|
||||
parentBlock1 = types.NewBlock(&parentHeader1, nil, nil, nil)
|
||||
parentBlock2 = types.NewBlock(&parentHeader2, nil, nil, nil)
|
||||
|
||||
parentHash1 = parentBlock1.Hash()
|
||||
parentHash2 = parentBlock2.Hash()
|
||||
|
||||
testRoot1 = common.HexToHash("0x03")
|
||||
testRoot2 = common.HexToHash("0x04")
|
||||
testRoot3 = common.HexToHash("0x04")
|
||||
header1 = types.Header{ParentHash: parentHash1, Root: testRoot1}
|
||||
header2 = types.Header{ParentHash: parentHash2, Root: testRoot2}
|
||||
header3 = types.Header{ParentHash: common.HexToHash("parent hash"), Root: testRoot3}
|
||||
|
||||
testBlock1 = types.NewBlock(&header1, nil, nil, nil)
|
||||
testBlock2 = types.NewBlock(&header2, nil, nil, nil)
|
||||
testBlock3 = types.NewBlock(&header3, nil, nil, nil)
|
||||
|
||||
event1 = core.ChainEvent{Block: testBlock1}
|
||||
event2 = core.ChainEvent{Block: testBlock2}
|
||||
event3 = core.ChainEvent{Block: testBlock3}
|
||||
)
|
||||
|
||||
func testErrorInChainEventLoop(t *testing.T) {
|
||||
//the first chain event causes and error (in blockchain mock)
|
||||
builder := mocks.Builder{}
|
||||
blockChain := mocks.BlockChain{}
|
||||
service := statediff.Service{
|
||||
Builder: &builder,
|
||||
BlockChain: &blockChain,
|
||||
QuitChan: make(chan bool),
|
||||
Subscriptions: make(map[rpc.ID]statediff.Subscription),
|
||||
}
|
||||
testRoot2 = common.HexToHash("0xTestRoot2")
|
||||
blockChain.SetParentBlocksToReturn([]*types.Block{parentBlock1, parentBlock2})
|
||||
blockChain.SetChainEvents([]core.ChainEvent{event1, event2, event3})
|
||||
service.Loop(eventsChannel)
|
||||
if !reflect.DeepEqual(builder.BlockHash, testBlock2.Hash()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.BlockHash, testBlock2.Hash())
|
||||
}
|
||||
if !bytes.Equal(builder.OldStateRoot.Bytes(), parentBlock2.Root().Bytes()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.OldStateRoot, parentBlock2.Root())
|
||||
}
|
||||
if !bytes.Equal(builder.NewStateRoot.Bytes(), testBlock2.Root().Bytes()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.NewStateRoot, testBlock2.Root())
|
||||
}
|
||||
//look up the parent block from its hash
|
||||
expectedHashes := []common.Hash{testBlock1.ParentHash(), testBlock2.ParentHash()}
|
||||
if !reflect.DeepEqual(blockChain.ParentHashesLookedUp, expectedHashes) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", blockChain.ParentHashesLookedUp, expectedHashes)
|
||||
}
|
||||
}
|
||||
|
||||
func testErrorInBlockLoop(t *testing.T) {
|
||||
//second block's parent block can't be found
|
||||
builder := mocks.Builder{}
|
||||
blockChain := mocks.BlockChain{}
|
||||
service := statediff.Service{
|
||||
Builder: &builder,
|
||||
BlockChain: &blockChain,
|
||||
QuitChan: make(chan bool),
|
||||
Subscriptions: make(map[rpc.ID]statediff.Subscription),
|
||||
}
|
||||
|
||||
blockChain.SetParentBlocksToReturn([]*types.Block{parentBlock1, nil})
|
||||
blockChain.SetChainEvents([]core.ChainEvent{event1, event2})
|
||||
service.Loop(eventsChannel)
|
||||
|
||||
if !bytes.Equal(builder.BlockHash.Bytes(), testBlock1.Hash().Bytes()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.BlockHash, testBlock1.Hash())
|
||||
}
|
||||
if !bytes.Equal(builder.OldStateRoot.Bytes(), parentBlock1.Root().Bytes()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.OldStateRoot, parentBlock1.Root())
|
||||
}
|
||||
if !bytes.Equal(builder.NewStateRoot.Bytes(), testBlock1.Root().Bytes()) {
|
||||
t.Error("Test failure:", t.Name())
|
||||
t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.NewStateRoot, testBlock1.Root())
|
||||
}
|
||||
}
|
83
statediff/testhelpers/helpers.go
Normal file
83
statediff/testhelpers/helpers.go
Normal file
@ -0,0 +1,83 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package testhelpers
|
||||
|
||||
import (
|
||||
"math/big"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/consensus/ethash"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/core/vm"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
)
|
||||
|
||||
// MakeChain creates a chain of n blocks starting at and including parent.
|
||||
// the returned hash chain is ordered head->parent. In addition, every 3rd block
|
||||
// contains a transaction and every 5th an uncle to allow testing correct block
|
||||
// reassembly.
|
||||
func MakeChain(n int, parent *types.Block) ([]common.Hash, map[common.Hash]*types.Block, *core.BlockChain) {
|
||||
blocks, _ := core.GenerateChain(params.TestChainConfig, parent, ethash.NewFaker(), Testdb, n, testChainGen)
|
||||
headers := make([]*types.Header, len(blocks))
|
||||
for i, block := range blocks {
|
||||
headers[i] = block.Header()
|
||||
}
|
||||
chain, _ := core.NewBlockChain(Testdb, nil, params.TestChainConfig, ethash.NewFaker(), vm.Config{}, nil)
|
||||
|
||||
hashes := make([]common.Hash, n+1)
|
||||
hashes[len(hashes)-1] = parent.Hash()
|
||||
blockm := make(map[common.Hash]*types.Block, n+1)
|
||||
blockm[parent.Hash()] = parent
|
||||
for i, b := range blocks {
|
||||
hashes[len(hashes)-i-2] = b.Hash()
|
||||
blockm[b.Hash()] = b
|
||||
}
|
||||
return hashes, blockm, chain
|
||||
}
|
||||
|
||||
func testChainGen(i int, block *core.BlockGen) {
|
||||
signer := types.HomesteadSigner{}
|
||||
switch i {
|
||||
case 0:
|
||||
// In block 1, the test bank sends account #1 some ether.
|
||||
tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(TestBankAddress), Account1Addr, big.NewInt(10000), params.TxGas, nil, nil), signer, TestBankKey)
|
||||
block.AddTx(tx)
|
||||
case 1:
|
||||
// In block 2, the test bank sends some more ether to account #1.
|
||||
// account1Addr passes it on to account #2.
|
||||
// account1Addr creates a test contract.
|
||||
tx1, _ := types.SignTx(types.NewTransaction(block.TxNonce(TestBankAddress), Account1Addr, big.NewInt(1000), params.TxGas, nil, nil), signer, TestBankKey)
|
||||
nonce := block.TxNonce(Account1Addr)
|
||||
tx2, _ := types.SignTx(types.NewTransaction(nonce, Account2Addr, big.NewInt(1000), params.TxGas, nil, nil), signer, Account1Key)
|
||||
nonce++
|
||||
tx3, _ := types.SignTx(types.NewContractCreation(nonce, big.NewInt(0), 1000000, big.NewInt(0), ContractCode), signer, Account1Key)
|
||||
ContractAddr = crypto.CreateAddress(Account1Addr, nonce) //0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476592
|
||||
block.AddTx(tx1)
|
||||
block.AddTx(tx2)
|
||||
block.AddTx(tx3)
|
||||
case 2:
|
||||
// Block 3 is empty but was mined by account #2.
|
||||
block.SetCoinbase(Account2Addr)
|
||||
//get function: 60cd2685
|
||||
//put function: c16431b9
|
||||
data := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003")
|
||||
tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(TestBankAddress), ContractAddr, big.NewInt(0), 100000, nil, data), signer, TestBankKey)
|
||||
block.AddTx(tx)
|
||||
}
|
||||
}
|
86
statediff/testhelpers/mocks/blockchain.go
Normal file
86
statediff/testhelpers/mocks/blockchain.go
Normal file
@ -0,0 +1,86 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mocks
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/event"
|
||||
)
|
||||
|
||||
// BlockChain is a mock blockchain for testing
|
||||
type BlockChain struct {
|
||||
ParentHashesLookedUp []common.Hash
|
||||
parentBlocksToReturn []*types.Block
|
||||
callCount int
|
||||
ChainEvents []core.ChainEvent
|
||||
}
|
||||
|
||||
// AddToStateDiffProcessedCollection mock method
|
||||
func (blockChain *BlockChain) AddToStateDiffProcessedCollection(hash common.Hash) {}
|
||||
|
||||
// SetParentBlocksToReturn mock method
|
||||
func (blockChain *BlockChain) SetParentBlocksToReturn(blocks []*types.Block) {
|
||||
blockChain.parentBlocksToReturn = blocks
|
||||
}
|
||||
|
||||
// GetBlockByHash mock method
|
||||
func (blockChain *BlockChain) GetBlockByHash(hash common.Hash) *types.Block {
|
||||
blockChain.ParentHashesLookedUp = append(blockChain.ParentHashesLookedUp, hash)
|
||||
|
||||
var parentBlock *types.Block
|
||||
if len(blockChain.parentBlocksToReturn) > 0 {
|
||||
parentBlock = blockChain.parentBlocksToReturn[blockChain.callCount]
|
||||
}
|
||||
|
||||
blockChain.callCount++
|
||||
return parentBlock
|
||||
}
|
||||
|
||||
// SetChainEvents mock method
|
||||
func (blockChain *BlockChain) SetChainEvents(chainEvents []core.ChainEvent) {
|
||||
blockChain.ChainEvents = chainEvents
|
||||
}
|
||||
|
||||
// SubscribeChainEvent mock method
|
||||
func (blockChain *BlockChain) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
|
||||
subErr := errors.New("Subscription Error")
|
||||
|
||||
var eventCounter int
|
||||
subscription := event.NewSubscription(func(quit <-chan struct{}) error {
|
||||
for _, chainEvent := range blockChain.ChainEvents {
|
||||
if eventCounter > 1 {
|
||||
time.Sleep(250 * time.Millisecond)
|
||||
return subErr
|
||||
}
|
||||
select {
|
||||
case ch <- chainEvent:
|
||||
case <-quit:
|
||||
return nil
|
||||
}
|
||||
eventCounter++
|
||||
}
|
||||
return nil
|
||||
})
|
||||
|
||||
return subscription
|
||||
}
|
52
statediff/testhelpers/mocks/builder.go
Normal file
52
statediff/testhelpers/mocks/builder.go
Normal file
@ -0,0 +1,52 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mocks
|
||||
|
||||
import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
)
|
||||
|
||||
// Builder is a mock state diff builder
|
||||
type Builder struct {
|
||||
OldStateRoot common.Hash
|
||||
NewStateRoot common.Hash
|
||||
BlockNumber int64
|
||||
BlockHash common.Hash
|
||||
stateDiff statediff.StateDiff
|
||||
builderError error
|
||||
}
|
||||
|
||||
// BuildStateDiff mock method
|
||||
func (builder *Builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (statediff.StateDiff, error) {
|
||||
builder.OldStateRoot = oldStateRoot
|
||||
builder.NewStateRoot = newStateRoot
|
||||
builder.BlockNumber = blockNumber
|
||||
builder.BlockHash = blockHash
|
||||
|
||||
return builder.stateDiff, builder.builderError
|
||||
}
|
||||
|
||||
// SetStateDiffToBuild mock method
|
||||
func (builder *Builder) SetStateDiffToBuild(stateDiff statediff.StateDiff) {
|
||||
builder.stateDiff = stateDiff
|
||||
}
|
||||
|
||||
// SetBuilderError mock method
|
||||
func (builder *Builder) SetBuilderError(err error) {
|
||||
builder.builderError = err
|
||||
}
|
36
statediff/testhelpers/mocks/publisher.go
Normal file
36
statediff/testhelpers/mocks/publisher.go
Normal file
@ -0,0 +1,36 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mocks
|
||||
|
||||
import "github.com/ethereum/go-ethereum/statediff"
|
||||
|
||||
// Publisher mock
|
||||
type Publisher struct {
|
||||
StateDiff *statediff.StateDiff
|
||||
publisherError error
|
||||
}
|
||||
|
||||
// PublishStateDiff mock method
|
||||
func (publisher *Publisher) PublishStateDiff(sd *statediff.StateDiff) (string, error) {
|
||||
publisher.StateDiff = sd
|
||||
return "", publisher.publisherError
|
||||
}
|
||||
|
||||
// SetPublisherError mock method
|
||||
func (publisher *Publisher) SetPublisherError(err error) {
|
||||
publisher.publisherError = err
|
||||
}
|
171
statediff/testhelpers/mocks/service.go
Normal file
171
statediff/testhelpers/mocks/service.go
Normal file
@ -0,0 +1,171 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mocks
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/p2p"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
)
|
||||
|
||||
// MockStateDiffService is a mock state diff service
|
||||
type MockStateDiffService struct {
|
||||
sync.Mutex
|
||||
Builder statediff.Builder
|
||||
ReturnProtocol []p2p.Protocol
|
||||
ReturnAPIs []rpc.API
|
||||
BlockChan chan *types.Block
|
||||
ParentBlockChan chan *types.Block
|
||||
QuitChan chan bool
|
||||
Subscriptions map[rpc.ID]statediff.Subscription
|
||||
}
|
||||
|
||||
// Protocols mock method
|
||||
func (sds *MockStateDiffService) Protocols() []p2p.Protocol {
|
||||
return []p2p.Protocol{}
|
||||
}
|
||||
|
||||
// APIs mock method
|
||||
func (sds *MockStateDiffService) APIs() []rpc.API {
|
||||
return []rpc.API{
|
||||
{
|
||||
Namespace: statediff.APIName,
|
||||
Version: statediff.APIVersion,
|
||||
Service: statediff.NewPublicStateDiffAPI(sds),
|
||||
Public: true,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Loop mock method
|
||||
func (sds *MockStateDiffService) Loop(chan core.ChainEvent) {
|
||||
//loop through chain events until no more
|
||||
HandleBlockChLoop:
|
||||
for {
|
||||
select {
|
||||
case block := <-sds.BlockChan:
|
||||
currentBlock := block
|
||||
parentBlock := <-sds.ParentBlockChan
|
||||
parentHash := parentBlock.Hash()
|
||||
if parentBlock == nil {
|
||||
log.Error("Parent block is nil, skipping this block",
|
||||
"parent block hash", parentHash.String(),
|
||||
"current block number", currentBlock.Number())
|
||||
break HandleBlockChLoop
|
||||
}
|
||||
|
||||
stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number().Int64(), currentBlock.Hash())
|
||||
if err != nil {
|
||||
log.Error("Error building statediff", "block number", currentBlock.Number(), "error", err)
|
||||
}
|
||||
rlpBuff := new(bytes.Buffer)
|
||||
currentBlock.EncodeRLP(rlpBuff)
|
||||
blockRlp := rlpBuff.Bytes()
|
||||
stateDiffRlp, _ := rlp.EncodeToBytes(stateDiff)
|
||||
payload := statediff.Payload{
|
||||
BlockRlp: blockRlp,
|
||||
StateDiffRlp: stateDiffRlp,
|
||||
Err: err,
|
||||
}
|
||||
// If we have any websocket subscription listening in, send the data to them
|
||||
sds.send(payload)
|
||||
case <-sds.QuitChan:
|
||||
log.Debug("Quitting the statediff block channel")
|
||||
sds.close()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Subscribe mock method
|
||||
func (sds *MockStateDiffService) Subscribe(id rpc.ID, sub chan<- statediff.Payload, quitChan chan<- bool) {
|
||||
log.Info("Subscribing to the statediff service")
|
||||
sds.Lock()
|
||||
sds.Subscriptions[id] = statediff.Subscription{
|
||||
PayloadChan: sub,
|
||||
QuitChan: quitChan,
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
||||
|
||||
// Unsubscribe mock method
|
||||
func (sds *MockStateDiffService) Unsubscribe(id rpc.ID) error {
|
||||
log.Info("Unsubscribing from the statediff service")
|
||||
sds.Lock()
|
||||
_, ok := sds.Subscriptions[id]
|
||||
if !ok {
|
||||
return fmt.Errorf("cannot unsubscribe; subscription for id %s does not exist", id)
|
||||
}
|
||||
delete(sds.Subscriptions, id)
|
||||
sds.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (sds *MockStateDiffService) send(payload statediff.Payload) {
|
||||
sds.Lock()
|
||||
for id, sub := range sds.Subscriptions {
|
||||
select {
|
||||
case sub.PayloadChan <- payload:
|
||||
log.Info("sending state diff payload to subscription %s", id)
|
||||
default:
|
||||
log.Info("unable to send payload to subscription %s; channel has no receiver", id)
|
||||
}
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
||||
|
||||
func (sds *MockStateDiffService) close() {
|
||||
sds.Lock()
|
||||
for id, sub := range sds.Subscriptions {
|
||||
select {
|
||||
case sub.QuitChan <- true:
|
||||
delete(sds.Subscriptions, id)
|
||||
log.Info("closing subscription %s", id)
|
||||
default:
|
||||
log.Info("unable to close subscription %s; channel has no receiver", id)
|
||||
}
|
||||
}
|
||||
sds.Unlock()
|
||||
}
|
||||
|
||||
// Start mock method
|
||||
func (sds *MockStateDiffService) Start(server *p2p.Server) error {
|
||||
log.Info("Starting statediff service")
|
||||
if sds.ParentBlockChan == nil || sds.BlockChan == nil {
|
||||
return errors.New("mock StateDiffingService requires preconfiguration with a MockParentBlockChan and MockBlockChan")
|
||||
}
|
||||
chainEventCh := make(chan core.ChainEvent, 10)
|
||||
go sds.Loop(chainEventCh)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop mock method
|
||||
func (sds *MockStateDiffService) Stop() error {
|
||||
log.Info("Stopping statediff service")
|
||||
close(sds.QuitChan)
|
||||
return nil
|
||||
}
|
127
statediff/testhelpers/mocks/service_test.go
Normal file
127
statediff/testhelpers/mocks/service_test.go
Normal file
@ -0,0 +1,127 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package mocks
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/big"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
"github.com/ethereum/go-ethereum/statediff/testhelpers"
|
||||
)
|
||||
|
||||
var block0, block1 *types.Block
|
||||
var burnLeafKey = testhelpers.AddressToLeafKey(common.HexToAddress("0x0"))
|
||||
var emptyAccountDiffEventualMap = make(statediff.AccountDiffsMap)
|
||||
var account1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: uint64(0),
|
||||
Balance: big.NewInt(10000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"),
|
||||
})
|
||||
var burnAccount1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: uint64(0),
|
||||
Balance: big.NewInt(2000000000000000000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"),
|
||||
})
|
||||
var bankAccount1, _ = rlp.EncodeToBytes(state.Account{
|
||||
Nonce: uint64(1),
|
||||
Balance: big.NewInt(testhelpers.TestBankFunds.Int64() - 10000),
|
||||
CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(),
|
||||
Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"),
|
||||
})
|
||||
|
||||
func TestAPI(t *testing.T) {
|
||||
_, blockMap, chain := testhelpers.MakeChain(3, testhelpers.Genesis)
|
||||
defer chain.Stop()
|
||||
block0Hash := common.HexToHash("0xd1721cfd0b29c36fd7a68f25c128e86413fb666a6e1d68e89b875bd299262661")
|
||||
block1Hash := common.HexToHash("0xbbe88de60ba33a3f18c0caa37d827bfb70252e19e40a07cd34041696c35ecb1a")
|
||||
block0 = blockMap[block0Hash]
|
||||
block1 = blockMap[block1Hash]
|
||||
blockChan := make(chan *types.Block)
|
||||
parentBlockChain := make(chan *types.Block)
|
||||
serviceQuitChan := make(chan bool)
|
||||
mockService := MockStateDiffService{
|
||||
Mutex: sync.Mutex{},
|
||||
Builder: statediff.NewBuilder(testhelpers.Testdb, chain),
|
||||
BlockChan: blockChan,
|
||||
ParentBlockChan: parentBlockChain,
|
||||
QuitChan: serviceQuitChan,
|
||||
Subscriptions: make(map[rpc.ID]statediff.Subscription),
|
||||
}
|
||||
mockService.Start(nil)
|
||||
id := rpc.NewID()
|
||||
payloadChan := make(chan statediff.Payload)
|
||||
quitChan := make(chan bool)
|
||||
mockService.Subscribe(id, payloadChan, quitChan)
|
||||
blockChan <- block1
|
||||
parentBlockChain <- block0
|
||||
expectedBlockRlp, _ := rlp.EncodeToBytes(block1)
|
||||
expectedStateDiff := &statediff.StateDiff{
|
||||
BlockNumber: block1.Number().Int64(),
|
||||
BlockHash: block1.Hash(),
|
||||
CreatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.Account1LeafKey: {
|
||||
Key: testhelpers.Account1LeafKey.Bytes(),
|
||||
Value: account1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 128, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
burnLeafKey: {
|
||||
Key: burnLeafKey.Bytes(),
|
||||
Value: burnAccount1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
DeletedAccounts: emptyAccountDiffEventualMap,
|
||||
UpdatedAccounts: statediff.AccountDiffsMap{
|
||||
testhelpers.BankLeafKey: {
|
||||
Key: testhelpers.BankLeafKey.Bytes(),
|
||||
Value: bankAccount1,
|
||||
Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128},
|
||||
{248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 1, 132, 5, 245, 185, 240, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}},
|
||||
Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16},
|
||||
Storage: []statediff.StorageDiff{},
|
||||
},
|
||||
},
|
||||
}
|
||||
expectedStateDiffRlp, _ := rlp.EncodeToBytes(expectedStateDiff)
|
||||
select {
|
||||
case payload := <-payloadChan:
|
||||
if !bytes.Equal(payload.BlockRlp, expectedBlockRlp) {
|
||||
t.Errorf("payload does not have expected block\r\actual: %v\r\nexpected: %v", payload.BlockRlp, expectedBlockRlp)
|
||||
}
|
||||
if !bytes.Equal(payload.StateDiffRlp, expectedStateDiffRlp) {
|
||||
t.Errorf("payload does not have expected state diff\r\actual: %v\r\nexpected: %v", payload.StateDiffRlp, expectedStateDiffRlp)
|
||||
}
|
||||
case <-quitChan:
|
||||
t.Errorf("channel quit before delivering payload")
|
||||
}
|
||||
}
|
114
statediff/testhelpers/test_data.go
Normal file
114
statediff/testhelpers/test_data.go
Normal file
@ -0,0 +1,114 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package testhelpers
|
||||
|
||||
import (
|
||||
"math/big"
|
||||
"math/rand"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/statediff"
|
||||
)
|
||||
|
||||
// AddressToLeafKey hashes an returns an address
|
||||
func AddressToLeafKey(address common.Address) common.Hash {
|
||||
return common.BytesToHash(crypto.Keccak256(address[:]))
|
||||
}
|
||||
|
||||
// Test variables
|
||||
var (
|
||||
BlockNumber = rand.Int63()
|
||||
BlockHash = "0xfa40fbe2d98d98b3363a778d52f2bcd29d6790b9b3f3cab2b167fd12d3550f73"
|
||||
CodeHash = common.Hex2Bytes("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470")
|
||||
NewNonceValue = rand.Uint64()
|
||||
NewBalanceValue = rand.Int63()
|
||||
ContractRoot = common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421")
|
||||
StoragePath = common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes()
|
||||
StorageKey = common.HexToHash("0000000000000000000000000000000000000000000000000000000000000001").Bytes()
|
||||
StorageValue = common.Hex2Bytes("0x03")
|
||||
storage = []statediff.StorageDiff{{
|
||||
Key: StorageKey,
|
||||
Value: StorageValue,
|
||||
Path: StoragePath,
|
||||
Proof: [][]byte{},
|
||||
}}
|
||||
emptyStorage = make([]statediff.StorageDiff, 0)
|
||||
address = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476592")
|
||||
ContractLeafKey = AddressToLeafKey(address)
|
||||
anotherAddress = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476593")
|
||||
AnotherContractLeafKey = AddressToLeafKey(anotherAddress)
|
||||
testAccount = state.Account{
|
||||
Nonce: NewNonceValue,
|
||||
Balance: big.NewInt(NewBalanceValue),
|
||||
Root: ContractRoot,
|
||||
CodeHash: CodeHash,
|
||||
}
|
||||
valueBytes, _ = rlp.EncodeToBytes(testAccount)
|
||||
CreatedAccountDiffs = statediff.AccountDiffsMap{
|
||||
ContractLeafKey: {
|
||||
Key: ContractLeafKey.Bytes(),
|
||||
Value: valueBytes,
|
||||
Storage: storage,
|
||||
},
|
||||
AnotherContractLeafKey: {
|
||||
Key: AnotherContractLeafKey.Bytes(),
|
||||
Value: valueBytes,
|
||||
Storage: emptyStorage,
|
||||
},
|
||||
}
|
||||
|
||||
UpdatedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: {
|
||||
Key: ContractLeafKey.Bytes(),
|
||||
Value: valueBytes,
|
||||
Storage: storage,
|
||||
}}
|
||||
|
||||
DeletedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: {
|
||||
Key: ContractLeafKey.Bytes(),
|
||||
Value: valueBytes,
|
||||
Storage: storage,
|
||||
}}
|
||||
|
||||
TestStateDiff = statediff.StateDiff{
|
||||
BlockNumber: BlockNumber,
|
||||
BlockHash: common.HexToHash(BlockHash),
|
||||
CreatedAccounts: CreatedAccountDiffs,
|
||||
DeletedAccounts: DeletedAccountDiffs,
|
||||
UpdatedAccounts: UpdatedAccountDiffs,
|
||||
}
|
||||
Testdb = rawdb.NewMemoryDatabase()
|
||||
|
||||
TestBankKey, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
|
||||
TestBankAddress = crypto.PubkeyToAddress(TestBankKey.PublicKey) //0x71562b71999873DB5b286dF957af199Ec94617F7
|
||||
BankLeafKey = AddressToLeafKey(TestBankAddress)
|
||||
TestBankFunds = big.NewInt(100000000)
|
||||
Genesis = core.GenesisBlockForTesting(Testdb, TestBankAddress, TestBankFunds)
|
||||
|
||||
Account1Key, _ = crypto.HexToECDSA("8a1f9a8f95be41cd7ccb6168179afb4504aefe388d1e14474d32c45c72ce7b7a")
|
||||
Account2Key, _ = crypto.HexToECDSA("49a7b37aa6f6645917e7b807e9d1c00d4fa71f18343b0d4122a4d2df64dd6fee")
|
||||
Account1Addr = crypto.PubkeyToAddress(Account1Key.PublicKey) //0x703c4b2bD70c169f5717101CaeE543299Fc946C7
|
||||
Account2Addr = crypto.PubkeyToAddress(Account2Key.PublicKey) //0x0D3ab14BBaD3D99F4203bd7a11aCB94882050E7e
|
||||
Account1LeafKey = AddressToLeafKey(Account1Addr)
|
||||
Account2LeafKey = AddressToLeafKey(Account2Addr)
|
||||
ContractCode = common.Hex2Bytes("608060405234801561001057600080fd5b50602060405190810160405280600160ff16815250600090600161003592919061003b565b506100a5565b826064810192821561006f579160200282015b8281111561006e578251829060ff1690559160200191906001019061004e565b5b50905061007c9190610080565b5090565b6100a291905b8082111561009e576000816000905550600101610086565b5090565b90565b610124806100b46000396000f3fe6080604052348015600f57600080fd5b5060043610604f576000357c01000000000000000000000000000000000000000000000000000000009004806360cd2685146054578063c16431b9146093575b600080fd5b607d60048036036020811015606857600080fd5b810190808035906020019092919050505060c8565b6040518082815260200191505060405180910390f35b60c66004803603604081101560a757600080fd5b81019080803590602001909291908035906020019092919050505060e0565b005b6000808260648110151560d757fe5b01549050919050565b8060008360648110151560ef57fe5b0181905550505056fea165627a7a7230582064e918c3140a117bf3aa65865a9b9e83fae21ad1720506e7933b2a9f54bb40260029")
|
||||
ContractAddr common.Address
|
||||
)
|
113
statediff/types.go
Normal file
113
statediff/types.go
Normal file
@ -0,0 +1,113 @@
|
||||
// Copyright 2015 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Contains a batch of utility type declarations used by the tests. As the node
|
||||
// operates on unique types, a lot of them are needed to check various features.
|
||||
|
||||
package statediff
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
)
|
||||
|
||||
// AccountsMap is a mapping of keccak256(address) => accountWrapper
|
||||
type AccountsMap map[common.Hash]accountWrapper
|
||||
|
||||
// AccountWrapper is used to temporary associate the unpacked account with its raw values
|
||||
type accountWrapper struct {
|
||||
Account state.Account
|
||||
RawKey []byte
|
||||
RawValue []byte
|
||||
Proof [][]byte
|
||||
Path []byte
|
||||
}
|
||||
|
||||
// StateDiff is the final output structure from the builder
|
||||
type StateDiff struct {
|
||||
BlockNumber int64 `json:"blockNumber" gencodec:"required"`
|
||||
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
|
||||
CreatedAccounts AccountDiffsMap `json:"createdAccounts" gencodec:"required"`
|
||||
DeletedAccounts AccountDiffsMap `json:"deletedAccounts" gencodec:"required"`
|
||||
UpdatedAccounts AccountDiffsMap `json:"updatedAccounts" gencodec:"required"`
|
||||
|
||||
encoded []byte
|
||||
err error
|
||||
}
|
||||
|
||||
func (sd *StateDiff) ensureEncoded() {
|
||||
if sd.encoded == nil && sd.err == nil {
|
||||
sd.encoded, sd.err = json.Marshal(sd)
|
||||
}
|
||||
}
|
||||
|
||||
// Length to implement Encoder interface for StateDiff
|
||||
func (sd *StateDiff) Length() int {
|
||||
sd.ensureEncoded()
|
||||
return len(sd.encoded)
|
||||
}
|
||||
|
||||
// Encode to implement Encoder interface for StateDiff
|
||||
func (sd *StateDiff) Encode() ([]byte, error) {
|
||||
sd.ensureEncoded()
|
||||
return sd.encoded, sd.err
|
||||
}
|
||||
|
||||
// AccountDiffsMap is a mapping of keccak256(address) => AccountDiff
|
||||
type AccountDiffsMap map[common.Hash]AccountDiff
|
||||
|
||||
// AccountDiff holds the data for a single state diff leaf node
|
||||
type AccountDiff struct {
|
||||
Key []byte `json:"key" gencodec:"required"`
|
||||
Value []byte `json:"value" gencodec:"required"`
|
||||
Proof [][]byte `json:"proof" gencodec:"required"`
|
||||
Storage []StorageDiff `json:"storage" gencodec:"required"`
|
||||
Path []byte `json:"path" gencodec:"required"`
|
||||
}
|
||||
|
||||
// StorageDiff holds the data for a single storage diff leaf node
|
||||
type StorageDiff struct {
|
||||
Key []byte `json:"key" gencodec:"required"`
|
||||
Value []byte `json:"value" gencodec:"required"`
|
||||
Proof [][]byte `json:"proof" gencodec:"required"`
|
||||
Path []byte `json:"path" gencodec:"required"`
|
||||
}
|
||||
|
||||
/*
|
||||
// State trie leaf is just a short node, below
|
||||
// that has an rlp encoded account as the value
|
||||
|
||||
|
||||
// SO each account diffs map is reall a map of shortnode keys to values
|
||||
// Flatten to a slice of short nodes?
|
||||
|
||||
// Need to coerce into:
|
||||
|
||||
type TrieNode struct {
|
||||
// leaf, extension or branch
|
||||
nodeKind string
|
||||
|
||||
// If leaf or extension: [0] is key, [1] is val.
|
||||
// If branch: [0] - [16] are children.
|
||||
elements []interface{}
|
||||
|
||||
// IPLD block information
|
||||
cid *cid.Cid
|
||||
rawdata []byte
|
||||
}
|
||||
*/
|
@ -76,9 +76,9 @@ func keybytesToHex(str []byte) []byte {
|
||||
return nibbles
|
||||
}
|
||||
|
||||
// hexToKeybytes turns hex nibbles into key bytes.
|
||||
// hexToKeyBytes turns hex nibbles into key bytes.
|
||||
// This can only be used for keys of even length.
|
||||
func hexToKeybytes(hex []byte) []byte {
|
||||
func hexToKeyBytes(hex []byte) []byte {
|
||||
if hasTerm(hex) {
|
||||
hex = hex[:len(hex)-1]
|
||||
}
|
||||
|
@ -69,8 +69,8 @@ func TestHexKeybytes(t *testing.T) {
|
||||
if h := keybytesToHex(test.key); !bytes.Equal(h, test.hexOut) {
|
||||
t.Errorf("keybytesToHex(%x) -> %x, want %x", test.key, h, test.hexOut)
|
||||
}
|
||||
if k := hexToKeybytes(test.hexIn); !bytes.Equal(k, test.key) {
|
||||
t.Errorf("hexToKeybytes(%x) -> %x, want %x", test.hexIn, k, test.key)
|
||||
if k := hexToKeyBytes(test.hexIn); !bytes.Equal(k, test.key) {
|
||||
t.Errorf("hexToKeyBytes(%x) -> %x, want %x", test.hexIn, k, test.key)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -99,6 +99,6 @@ func BenchmarkKeybytesToHex(b *testing.B) {
|
||||
func BenchmarkHexToKeybytes(b *testing.B) {
|
||||
testBytes := []byte{7, 6, 6, 5, 7, 2, 6, 2, 16}
|
||||
for i := 0; i < b.N; i++ {
|
||||
hexToKeybytes(testBytes)
|
||||
hexToKeyBytes(testBytes)
|
||||
}
|
||||
}
|
||||
|
@ -162,7 +162,7 @@ func (it *nodeIterator) Leaf() bool {
|
||||
func (it *nodeIterator) LeafKey() []byte {
|
||||
if len(it.stack) > 0 {
|
||||
if _, ok := it.stack[len(it.stack)-1].node.(valueNode); ok {
|
||||
return hexToKeybytes(it.path)
|
||||
return hexToKeyBytes(it.path)
|
||||
}
|
||||
}
|
||||
panic("not at leaf")
|
||||
|
125
vendor/github.com/onsi/gomega/CHANGELOG.md
generated
vendored
Normal file
125
vendor/github.com/onsi/gomega/CHANGELOG.md
generated
vendored
Normal file
@ -0,0 +1,125 @@
|
||||
## 1.4.3
|
||||
|
||||
### Fixes:
|
||||
|
||||
- ensure file name and line numbers are correctly reported for XUnit [6fff58f]
|
||||
- Fixed matcher for content-type (#305) [69d9b43]
|
||||
|
||||
## 1.4.2
|
||||
|
||||
### Fixes:
|
||||
|
||||
- Add go.mod and go.sum files to define the gomega go module [f3de367, a085d30]
|
||||
- Work around go vet issue with Go v1.11 (#300) [40dd6ad]
|
||||
- Better output when using with go XUnit-style tests, fixes #255 (#297) [29a4b97]
|
||||
- Fix MatchJSON fail to parse json.RawMessage (#298) [ae19f1b]
|
||||
- show threshold in failure message of BeNumericallyMatcher (#293) [4bbecc8]
|
||||
|
||||
## 1.4.1
|
||||
|
||||
### Fixes:
|
||||
|
||||
- Update documentation formatting and examples (#289) [9be8410]
|
||||
- allow 'Receive' matcher to be used with concrete types (#286) [41673fd]
|
||||
- Fix data race in ghttp server (#283) [7ac6b01]
|
||||
- Travis badge should only show master [cc102ab]
|
||||
|
||||
## 1.4.0
|
||||
|
||||
### Features
|
||||
- Make string pretty diff user configurable (#273) [eb112ce, 649b44d]
|
||||
|
||||
### Fixes
|
||||
- Use httputil.DumpRequest to pretty-print unhandled requests (#278) [a4ff0fc, b7d1a52]
|
||||
- fix typo floa32 > float32 (#272) [041ae3b, 6e33911]
|
||||
- Fix link to documentation on adding your own matchers (#270) [bb2c830, fcebc62]
|
||||
- Use setters and getters to avoid race condition (#262) [13057c3, a9c79f1]
|
||||
- Avoid sending a signal if the process is not alive (#259) [b8043e5, 4fc1762]
|
||||
- Improve message from AssignableToTypeOf when expected value is nil (#281) [9c1fb20]
|
||||
|
||||
## 1.3.0
|
||||
|
||||
Improvements:
|
||||
|
||||
- The `Equal` matcher matches byte slices more performantly.
|
||||
- Improved how `MatchError` matches error strings.
|
||||
- `MatchXML` ignores the order of xml node attributes.
|
||||
- Improve support for XUnit style golang tests. ([#254](https://github.com/onsi/gomega/issues/254))
|
||||
|
||||
Bug Fixes:
|
||||
|
||||
- Diff generation now handles multi-byte sequences correctly.
|
||||
- Multiple goroutines can now call `gexec.Build` concurrently.
|
||||
|
||||
## 1.2.0
|
||||
|
||||
Improvements:
|
||||
|
||||
- Added `BeSent` which attempts to send a value down a channel and fails if the attempt blocks. Can be paired with `Eventually` to safely send a value down a channel with a timeout.
|
||||
- `Ω`, `Expect`, `Eventually`, and `Consistently` now immediately `panic` if there is no registered fail handler. This is always a mistake that can hide failing tests.
|
||||
- `Receive()` no longer errors when passed a closed channel, it's perfectly fine to attempt to read from a closed channel so Ω(c).Should(Receive()) always fails and Ω(c).ShoudlNot(Receive()) always passes with a closed channel.
|
||||
- Added `HavePrefix` and `HaveSuffix` matchers.
|
||||
- `ghttp` can now handle concurrent requests.
|
||||
- Added `Succeed` which allows one to write `Ω(MyFunction()).Should(Succeed())`.
|
||||
- Improved `ghttp`'s behavior around failing assertions and panics:
|
||||
- If a registered handler makes a failing assertion `ghttp` will return `500`.
|
||||
- If a registered handler panics, `ghttp` will return `500` *and* fail the test. This is new behavior that may cause existing code to break. This code is almost certainly incorrect and creating a false positive.
|
||||
- `ghttp` servers can take an `io.Writer`. `ghttp` will write a line to the writer when each request arrives.
|
||||
- Added `WithTransform` matcher to allow munging input data before feeding into the relevant matcher
|
||||
- Added boolean `And`, `Or`, and `Not` matchers to allow creating composite matchers
|
||||
- Added `gbytes.TimeoutCloser`, `gbytes.TimeoutReader`, and `gbytes.TimeoutWriter` - these are convenience wrappers that timeout if the underlying Closer/Reader/Writer does not return within the alloted time.
|
||||
- Added `gbytes.BufferReader` - this constructs a `gbytes.Buffer` that asynchronously reads the passed-in `io.Reader` into its buffer.
|
||||
|
||||
Bug Fixes:
|
||||
- gexec: `session.Wait` now uses `EventuallyWithOffset` to get the right line number in the failure.
|
||||
- `ContainElement` no longer bails if a passed-in matcher errors.
|
||||
|
||||
## 1.0 (8/2/2014)
|
||||
|
||||
No changes. Dropping "beta" from the version number.
|
||||
|
||||
## 1.0.0-beta (7/8/2014)
|
||||
Breaking Changes:
|
||||
|
||||
- Changed OmegaMatcher interface. Instead of having `Match` return failure messages, two new methods `FailureMessage` and `NegatedFailureMessage` are called instead.
|
||||
- Moved and renamed OmegaFailHandler to types.GomegaFailHandler and OmegaMatcher to types.GomegaMatcher. Any references to OmegaMatcher in any custom matchers will need to be changed to point to types.GomegaMatcher
|
||||
|
||||
New Test-Support Features:
|
||||
|
||||
- `ghttp`: supports testing http clients
|
||||
- Provides a flexible fake http server
|
||||
- Provides a collection of chainable http handlers that perform assertions.
|
||||
- `gbytes`: supports making ordered assertions against streams of data
|
||||
- Provides a `gbytes.Buffer`
|
||||
- Provides a `Say` matcher to perform ordered assertions against output data
|
||||
- `gexec`: supports testing external processes
|
||||
- Provides support for building Go binaries
|
||||
- Wraps and starts `exec.Cmd` commands
|
||||
- Makes it easy to assert against stdout and stderr
|
||||
- Makes it easy to send signals and wait for processes to exit
|
||||
- Provides an `Exit` matcher to assert against exit code.
|
||||
|
||||
DSL Changes:
|
||||
|
||||
- `Eventually` and `Consistently` can accept `time.Duration` interval and polling inputs.
|
||||
- The default timeouts for `Eventually` and `Consistently` are now configurable.
|
||||
|
||||
New Matchers:
|
||||
|
||||
- `ConsistOf`: order-independent assertion against the elements of an array/slice or keys of a map.
|
||||
- `BeTemporally`: like `BeNumerically` but for `time.Time`
|
||||
- `HaveKeyWithValue`: asserts a map has a given key with the given value.
|
||||
|
||||
Updated Matchers:
|
||||
|
||||
- `Receive` matcher can take a matcher as an argument and passes only if the channel under test receives an objet that satisfies the passed-in matcher.
|
||||
- Matchers that implement `MatchMayChangeInTheFuture(actual interface{}) bool` can inform `Eventually` and/or `Consistently` when a match has no chance of changing status in the future. For example, `Receive` returns `false` when a channel is closed.
|
||||
|
||||
Misc:
|
||||
|
||||
- Start using semantic versioning
|
||||
- Start maintaining changelog
|
||||
|
||||
Major refactor:
|
||||
|
||||
- Pull out Gomega's internal to `internal`
|
14
vendor/github.com/onsi/gomega/CONTRIBUTING.md
generated
vendored
Normal file
14
vendor/github.com/onsi/gomega/CONTRIBUTING.md
generated
vendored
Normal file
@ -0,0 +1,14 @@
|
||||
# Contributing to Gomega
|
||||
|
||||
Your contributions to Gomega are essential for its long-term maintenance and improvement. To make a contribution:
|
||||
|
||||
- Please **open an issue first** - describe what problem you are trying to solve and give the community a forum for input and feedback ahead of investing time in writing code!
|
||||
- Ensure adequate test coverage:
|
||||
- Make sure to add appropriate unit tests
|
||||
- Please run all tests locally (`ginkgo -r -p`) and make sure they go green before submitting the PR
|
||||
- Please run following linter locally `go vet ./...` and make sure output does not contain any warnings
|
||||
- Update the documentation. In addition to standard `godoc` comments Gomega has extensive documentation on the `gh-pages` branch. If relevant, please submit a docs PR to that branch alongside your code PR.
|
||||
|
||||
If you're a committer, check out RELEASING.md to learn how to cut a release.
|
||||
|
||||
Thanks for supporting Gomega!
|
20
vendor/github.com/onsi/gomega/LICENSE
generated
vendored
Normal file
20
vendor/github.com/onsi/gomega/LICENSE
generated
vendored
Normal file
@ -0,0 +1,20 @@
|
||||
Copyright (c) 2013-2014 Onsi Fakhouri
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining
|
||||
a copy of this software and associated documentation files (the
|
||||
"Software"), to deal in the Software without restriction, including
|
||||
without limitation the rights to use, copy, modify, merge, publish,
|
||||
distribute, sublicense, and/or sell copies of the Software, and to
|
||||
permit persons to whom the Software is furnished to do so, subject to
|
||||
the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be
|
||||
included in all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
|
||||
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
|
||||
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
|
||||
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
|
||||
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
6
vendor/github.com/onsi/gomega/Makefile
generated
vendored
Normal file
6
vendor/github.com/onsi/gomega/Makefile
generated
vendored
Normal file
@ -0,0 +1,6 @@
|
||||
test:
|
||||
[ -z "`gofmt -s -w -l -e .`" ]
|
||||
go vet
|
||||
ginkgo -p -r --randomizeAllSpecs --failOnPending --randomizeSuites --race
|
||||
|
||||
.PHONY: test
|
21
vendor/github.com/onsi/gomega/README.md
generated
vendored
Normal file
21
vendor/github.com/onsi/gomega/README.md
generated
vendored
Normal file
@ -0,0 +1,21 @@
|
||||
![Gomega: Ginkgo's Preferred Matcher Library](http://onsi.github.io/gomega/images/gomega.png)
|
||||
|
||||
[![Build Status](https://travis-ci.org/onsi/gomega.svg?branch=master)](https://travis-ci.org/onsi/gomega)
|
||||
|
||||
Jump straight to the [docs](http://onsi.github.io/gomega/) to learn about Gomega, including a list of [all available matchers](http://onsi.github.io/gomega/#provided-matchers).
|
||||
|
||||
If you have a question, comment, bug report, feature request, etc. please open a GitHub issue.
|
||||
|
||||
## [Ginkgo](http://github.com/onsi/ginkgo): a BDD Testing Framework for Golang
|
||||
|
||||
Learn more about Ginkgo [here](http://onsi.github.io/ginkgo/)
|
||||
|
||||
## Community Matchers
|
||||
|
||||
A collection of community matchers is available on the [wiki](https://github.com/onsi/gomega/wiki).
|
||||
|
||||
## License
|
||||
|
||||
Gomega is MIT-Licensed
|
||||
|
||||
The `ConsistOf` matcher uses [goraph](https://github.com/amitkgupta/goraph) which is embedded in the source to simplify distribution. goraph has an MIT license.
|
12
vendor/github.com/onsi/gomega/RELEASING.md
generated
vendored
Normal file
12
vendor/github.com/onsi/gomega/RELEASING.md
generated
vendored
Normal file
@ -0,0 +1,12 @@
|
||||
A Gomega release is a tagged sha and a GitHub release. To cut a release:
|
||||
|
||||
1. Ensure CHANGELOG.md is up to date.
|
||||
- Use `git log --pretty=format:'- %s [%h]' HEAD...vX.X.X` to list all the commits since the last release
|
||||
- Categorize the changes into
|
||||
- Breaking Changes (requires a major version)
|
||||
- New Features (minor version)
|
||||
- Fixes (fix version)
|
||||
- Maintenance (which in general should not be mentioned in `CHANGELOG.md` as they have no user impact)
|
||||
2. Update GOMEGA_VERSION in `gomega_dsl.go`
|
||||
3. Push a commit with the version number as the commit message (e.g. `v1.3.0`)
|
||||
4. Create a new [GitHub release](https://help.github.com/articles/creating-releases/) with the version number as the tag (e.g. `v1.3.0`). List the key changes in the release notes.
|
382
vendor/github.com/onsi/gomega/format/format.go
generated
vendored
Normal file
382
vendor/github.com/onsi/gomega/format/format.go
generated
vendored
Normal file
@ -0,0 +1,382 @@
|
||||
/*
|
||||
Gomega's format package pretty-prints objects. It explores input objects recursively and generates formatted, indented output with type information.
|
||||
*/
|
||||
package format
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Use MaxDepth to set the maximum recursion depth when printing deeply nested objects
|
||||
var MaxDepth = uint(10)
|
||||
|
||||
/*
|
||||
By default, all objects (even those that implement fmt.Stringer and fmt.GoStringer) are recursively inspected to generate output.
|
||||
|
||||
Set UseStringerRepresentation = true to use GoString (for fmt.GoStringers) or String (for fmt.Stringer) instead.
|
||||
|
||||
Note that GoString and String don't always have all the information you need to understand why a test failed!
|
||||
*/
|
||||
var UseStringerRepresentation = false
|
||||
|
||||
/*
|
||||
Print the content of context objects. By default it will be suppressed.
|
||||
|
||||
Set PrintContextObjects = true to enable printing of the context internals.
|
||||
*/
|
||||
var PrintContextObjects = false
|
||||
|
||||
// TruncatedDiff choose if we should display a truncated pretty diff or not
|
||||
var TruncatedDiff = true
|
||||
|
||||
// Ctx interface defined here to keep backwards compatability with go < 1.7
|
||||
// It matches the context.Context interface
|
||||
type Ctx interface {
|
||||
Deadline() (deadline time.Time, ok bool)
|
||||
Done() <-chan struct{}
|
||||
Err() error
|
||||
Value(key interface{}) interface{}
|
||||
}
|
||||
|
||||
var contextType = reflect.TypeOf((*Ctx)(nil)).Elem()
|
||||
var timeType = reflect.TypeOf(time.Time{})
|
||||
|
||||
//The default indentation string emitted by the format package
|
||||
var Indent = " "
|
||||
|
||||
var longFormThreshold = 20
|
||||
|
||||
/*
|
||||
Generates a formatted matcher success/failure message of the form:
|
||||
|
||||
Expected
|
||||
<pretty printed actual>
|
||||
<message>
|
||||
<pretty printed expected>
|
||||
|
||||
If expected is omited, then the message looks like:
|
||||
|
||||
Expected
|
||||
<pretty printed actual>
|
||||
<message>
|
||||
*/
|
||||
func Message(actual interface{}, message string, expected ...interface{}) string {
|
||||
if len(expected) == 0 {
|
||||
return fmt.Sprintf("Expected\n%s\n%s", Object(actual, 1), message)
|
||||
}
|
||||
return fmt.Sprintf("Expected\n%s\n%s\n%s", Object(actual, 1), message, Object(expected[0], 1))
|
||||
}
|
||||
|
||||
/*
|
||||
|
||||
Generates a nicely formatted matcher success / failure message
|
||||
|
||||
Much like Message(...), but it attempts to pretty print diffs in strings
|
||||
|
||||
Expected
|
||||
<string>: "...aaaaabaaaaa..."
|
||||
to equal |
|
||||
<string>: "...aaaaazaaaaa..."
|
||||
|
||||
*/
|
||||
|
||||
func MessageWithDiff(actual, message, expected string) string {
|
||||
if TruncatedDiff && len(actual) >= truncateThreshold && len(expected) >= truncateThreshold {
|
||||
diffPoint := findFirstMismatch(actual, expected)
|
||||
formattedActual := truncateAndFormat(actual, diffPoint)
|
||||
formattedExpected := truncateAndFormat(expected, diffPoint)
|
||||
|
||||
spacesBeforeFormattedMismatch := findFirstMismatch(formattedActual, formattedExpected)
|
||||
|
||||
tabLength := 4
|
||||
spaceFromMessageToActual := tabLength + len("<string>: ") - len(message)
|
||||
padding := strings.Repeat(" ", spaceFromMessageToActual+spacesBeforeFormattedMismatch) + "|"
|
||||
return Message(formattedActual, message+padding, formattedExpected)
|
||||
}
|
||||
return Message(actual, message, expected)
|
||||
}
|
||||
|
||||
func truncateAndFormat(str string, index int) string {
|
||||
leftPadding := `...`
|
||||
rightPadding := `...`
|
||||
|
||||
start := index - charactersAroundMismatchToInclude
|
||||
if start < 0 {
|
||||
start = 0
|
||||
leftPadding = ""
|
||||
}
|
||||
|
||||
// slice index must include the mis-matched character
|
||||
lengthOfMismatchedCharacter := 1
|
||||
end := index + charactersAroundMismatchToInclude + lengthOfMismatchedCharacter
|
||||
if end > len(str) {
|
||||
end = len(str)
|
||||
rightPadding = ""
|
||||
|
||||
}
|
||||
return fmt.Sprintf("\"%s\"", leftPadding+str[start:end]+rightPadding)
|
||||
}
|
||||
|
||||
func findFirstMismatch(a, b string) int {
|
||||
aSlice := strings.Split(a, "")
|
||||
bSlice := strings.Split(b, "")
|
||||
|
||||
for index, str := range aSlice {
|
||||
if index > len(bSlice)-1 {
|
||||
return index
|
||||
}
|
||||
if str != bSlice[index] {
|
||||
return index
|
||||
}
|
||||
}
|
||||
|
||||
if len(b) > len(a) {
|
||||
return len(a) + 1
|
||||
}
|
||||
|
||||
return 0
|
||||
}
|
||||
|
||||
const (
|
||||
truncateThreshold = 50
|
||||
charactersAroundMismatchToInclude = 5
|
||||
)
|
||||
|
||||
/*
|
||||
Pretty prints the passed in object at the passed in indentation level.
|
||||
|
||||
Object recurses into deeply nested objects emitting pretty-printed representations of their components.
|
||||
|
||||
Modify format.MaxDepth to control how deep the recursion is allowed to go
|
||||
Set format.UseStringerRepresentation to true to return object.GoString() or object.String() when available instead of
|
||||
recursing into the object.
|
||||
|
||||
Set PrintContextObjects to true to print the content of objects implementing context.Context
|
||||
*/
|
||||
func Object(object interface{}, indentation uint) string {
|
||||
indent := strings.Repeat(Indent, int(indentation))
|
||||
value := reflect.ValueOf(object)
|
||||
return fmt.Sprintf("%s<%s>: %s", indent, formatType(object), formatValue(value, indentation))
|
||||
}
|
||||
|
||||
/*
|
||||
IndentString takes a string and indents each line by the specified amount.
|
||||
*/
|
||||
func IndentString(s string, indentation uint) string {
|
||||
components := strings.Split(s, "\n")
|
||||
result := ""
|
||||
indent := strings.Repeat(Indent, int(indentation))
|
||||
for i, component := range components {
|
||||
result += indent + component
|
||||
if i < len(components)-1 {
|
||||
result += "\n"
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func formatType(object interface{}) string {
|
||||
t := reflect.TypeOf(object)
|
||||
if t == nil {
|
||||
return "nil"
|
||||
}
|
||||
switch t.Kind() {
|
||||
case reflect.Chan:
|
||||
v := reflect.ValueOf(object)
|
||||
return fmt.Sprintf("%T | len:%d, cap:%d", object, v.Len(), v.Cap())
|
||||
case reflect.Ptr:
|
||||
return fmt.Sprintf("%T | %p", object, object)
|
||||
case reflect.Slice:
|
||||
v := reflect.ValueOf(object)
|
||||
return fmt.Sprintf("%T | len:%d, cap:%d", object, v.Len(), v.Cap())
|
||||
case reflect.Map:
|
||||
v := reflect.ValueOf(object)
|
||||
return fmt.Sprintf("%T | len:%d", object, v.Len())
|
||||
default:
|
||||
return fmt.Sprintf("%T", object)
|
||||
}
|
||||
}
|
||||
|
||||
func formatValue(value reflect.Value, indentation uint) string {
|
||||
if indentation > MaxDepth {
|
||||
return "..."
|
||||
}
|
||||
|
||||
if isNilValue(value) {
|
||||
return "nil"
|
||||
}
|
||||
|
||||
if UseStringerRepresentation {
|
||||
if value.CanInterface() {
|
||||
obj := value.Interface()
|
||||
switch x := obj.(type) {
|
||||
case fmt.GoStringer:
|
||||
return x.GoString()
|
||||
case fmt.Stringer:
|
||||
return x.String()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if !PrintContextObjects {
|
||||
if value.Type().Implements(contextType) && indentation > 1 {
|
||||
return "<suppressed context>"
|
||||
}
|
||||
}
|
||||
|
||||
switch value.Kind() {
|
||||
case reflect.Bool:
|
||||
return fmt.Sprintf("%v", value.Bool())
|
||||
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
|
||||
return fmt.Sprintf("%v", value.Int())
|
||||
case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64:
|
||||
return fmt.Sprintf("%v", value.Uint())
|
||||
case reflect.Uintptr:
|
||||
return fmt.Sprintf("0x%x", value.Uint())
|
||||
case reflect.Float32, reflect.Float64:
|
||||
return fmt.Sprintf("%v", value.Float())
|
||||
case reflect.Complex64, reflect.Complex128:
|
||||
return fmt.Sprintf("%v", value.Complex())
|
||||
case reflect.Chan:
|
||||
return fmt.Sprintf("0x%x", value.Pointer())
|
||||
case reflect.Func:
|
||||
return fmt.Sprintf("0x%x", value.Pointer())
|
||||
case reflect.Ptr:
|
||||
return formatValue(value.Elem(), indentation)
|
||||
case reflect.Slice:
|
||||
return formatSlice(value, indentation)
|
||||
case reflect.String:
|
||||
return formatString(value.String(), indentation)
|
||||
case reflect.Array:
|
||||
return formatSlice(value, indentation)
|
||||
case reflect.Map:
|
||||
return formatMap(value, indentation)
|
||||
case reflect.Struct:
|
||||
if value.Type() == timeType && value.CanInterface() {
|
||||
t, _ := value.Interface().(time.Time)
|
||||
return t.Format(time.RFC3339Nano)
|
||||
}
|
||||
return formatStruct(value, indentation)
|
||||
case reflect.Interface:
|
||||
return formatValue(value.Elem(), indentation)
|
||||
default:
|
||||
if value.CanInterface() {
|
||||
return fmt.Sprintf("%#v", value.Interface())
|
||||
}
|
||||
return fmt.Sprintf("%#v", value)
|
||||
}
|
||||
}
|
||||
|
||||
func formatString(object interface{}, indentation uint) string {
|
||||
if indentation == 1 {
|
||||
s := fmt.Sprintf("%s", object)
|
||||
components := strings.Split(s, "\n")
|
||||
result := ""
|
||||
for i, component := range components {
|
||||
if i == 0 {
|
||||
result += component
|
||||
} else {
|
||||
result += Indent + component
|
||||
}
|
||||
if i < len(components)-1 {
|
||||
result += "\n"
|
||||
}
|
||||
}
|
||||
|
||||
return fmt.Sprintf("%s", result)
|
||||
} else {
|
||||
return fmt.Sprintf("%q", object)
|
||||
}
|
||||
}
|
||||
|
||||
func formatSlice(v reflect.Value, indentation uint) string {
|
||||
if v.Kind() == reflect.Slice && v.Type().Elem().Kind() == reflect.Uint8 && isPrintableString(string(v.Bytes())) {
|
||||
return formatString(v.Bytes(), indentation)
|
||||
}
|
||||
|
||||
l := v.Len()
|
||||
result := make([]string, l)
|
||||
longest := 0
|
||||
for i := 0; i < l; i++ {
|
||||
result[i] = formatValue(v.Index(i), indentation+1)
|
||||
if len(result[i]) > longest {
|
||||
longest = len(result[i])
|
||||
}
|
||||
}
|
||||
|
||||
if longest > longFormThreshold {
|
||||
indenter := strings.Repeat(Indent, int(indentation))
|
||||
return fmt.Sprintf("[\n%s%s,\n%s]", indenter+Indent, strings.Join(result, ",\n"+indenter+Indent), indenter)
|
||||
}
|
||||
return fmt.Sprintf("[%s]", strings.Join(result, ", "))
|
||||
}
|
||||
|
||||
func formatMap(v reflect.Value, indentation uint) string {
|
||||
l := v.Len()
|
||||
result := make([]string, l)
|
||||
|
||||
longest := 0
|
||||
for i, key := range v.MapKeys() {
|
||||
value := v.MapIndex(key)
|
||||
result[i] = fmt.Sprintf("%s: %s", formatValue(key, indentation+1), formatValue(value, indentation+1))
|
||||
if len(result[i]) > longest {
|
||||
longest = len(result[i])
|
||||
}
|
||||
}
|
||||
|
||||
if longest > longFormThreshold {
|
||||
indenter := strings.Repeat(Indent, int(indentation))
|
||||
return fmt.Sprintf("{\n%s%s,\n%s}", indenter+Indent, strings.Join(result, ",\n"+indenter+Indent), indenter)
|
||||
}
|
||||
return fmt.Sprintf("{%s}", strings.Join(result, ", "))
|
||||
}
|
||||
|
||||
func formatStruct(v reflect.Value, indentation uint) string {
|
||||
t := v.Type()
|
||||
|
||||
l := v.NumField()
|
||||
result := []string{}
|
||||
longest := 0
|
||||
for i := 0; i < l; i++ {
|
||||
structField := t.Field(i)
|
||||
fieldEntry := v.Field(i)
|
||||
representation := fmt.Sprintf("%s: %s", structField.Name, formatValue(fieldEntry, indentation+1))
|
||||
result = append(result, representation)
|
||||
if len(representation) > longest {
|
||||
longest = len(representation)
|
||||
}
|
||||
}
|
||||
if longest > longFormThreshold {
|
||||
indenter := strings.Repeat(Indent, int(indentation))
|
||||
return fmt.Sprintf("{\n%s%s,\n%s}", indenter+Indent, strings.Join(result, ",\n"+indenter+Indent), indenter)
|
||||
}
|
||||
return fmt.Sprintf("{%s}", strings.Join(result, ", "))
|
||||
}
|
||||
|
||||
func isNilValue(a reflect.Value) bool {
|
||||
switch a.Kind() {
|
||||
case reflect.Invalid:
|
||||
return true
|
||||
case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice:
|
||||
return a.IsNil()
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
/*
|
||||
Returns true when the string is entirely made of printable runes, false otherwise.
|
||||
*/
|
||||
func isPrintableString(str string) bool {
|
||||
for _, runeValue := range str {
|
||||
if !strconv.IsPrint(runeValue) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
15
vendor/github.com/onsi/gomega/go.mod
generated
vendored
Normal file
15
vendor/github.com/onsi/gomega/go.mod
generated
vendored
Normal file
@ -0,0 +1,15 @@
|
||||
module github.com/onsi/gomega
|
||||
|
||||
require (
|
||||
github.com/fsnotify/fsnotify v1.4.7 // indirect
|
||||
github.com/golang/protobuf v1.2.0
|
||||
github.com/hpcloud/tail v1.0.0 // indirect
|
||||
github.com/onsi/ginkgo v1.6.0
|
||||
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd
|
||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f // indirect
|
||||
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e // indirect
|
||||
golang.org/x/text v0.3.0 // indirect
|
||||
gopkg.in/fsnotify.v1 v1.4.7 // indirect
|
||||
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
|
||||
gopkg.in/yaml.v2 v2.2.1
|
||||
)
|
24
vendor/github.com/onsi/gomega/go.sum
generated
vendored
Normal file
24
vendor/github.com/onsi/gomega/go.sum
generated
vendored
Normal file
@ -0,0 +1,24 @@
|
||||
github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I=
|
||||
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
|
||||
github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM=
|
||||
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||
github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
|
||||
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
|
||||
github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw=
|
||||
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
||||
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd h1:nTDtHvHSdCn1m6ITfMRqtOd/9+7a3s8RBNOZ3eYZzJA=
|
||||
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f h1:wMNYb4v58l5UBM7MYRLPG6ZhfOqbKu7X5eyFl8ZhKvA=
|
||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e h1:o3PsSEY8E4eXWkXrIP9YJALUkVZqzHJT5DOasTyn8Vs=
|
||||
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
|
||||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
|
||||
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
|
||||
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
|
||||
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
|
||||
gopkg.in/yaml.v2 v2.2.1 h1:mUhvW9EsL+naU5Q3cakzfE91YhliOondGd6ZrsDBHQE=
|
||||
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
429
vendor/github.com/onsi/gomega/gomega_dsl.go
generated
vendored
Normal file
429
vendor/github.com/onsi/gomega/gomega_dsl.go
generated
vendored
Normal file
@ -0,0 +1,429 @@
|
||||
/*
|
||||
Gomega is the Ginkgo BDD-style testing framework's preferred matcher library.
|
||||
|
||||
The godoc documentation describes Gomega's API. More comprehensive documentation (with examples!) is available at http://onsi.github.io/gomega/
|
||||
|
||||
Gomega on Github: http://github.com/onsi/gomega
|
||||
|
||||
Learn more about Ginkgo online: http://onsi.github.io/ginkgo
|
||||
|
||||
Ginkgo on Github: http://github.com/onsi/ginkgo
|
||||
|
||||
Gomega is MIT-Licensed
|
||||
*/
|
||||
package gomega
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/onsi/gomega/internal/assertion"
|
||||
"github.com/onsi/gomega/internal/asyncassertion"
|
||||
"github.com/onsi/gomega/internal/testingtsupport"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
const GOMEGA_VERSION = "1.4.3"
|
||||
|
||||
const nilFailHandlerPanic = `You are trying to make an assertion, but Gomega's fail handler is nil.
|
||||
If you're using Ginkgo then you probably forgot to put your assertion in an It().
|
||||
Alternatively, you may have forgotten to register a fail handler with RegisterFailHandler() or RegisterTestingT().
|
||||
Depending on your vendoring solution you may be inadvertently importing gomega and subpackages (e.g. ghhtp, gexec,...) from different locations.
|
||||
`
|
||||
|
||||
var globalFailWrapper *types.GomegaFailWrapper
|
||||
|
||||
var defaultEventuallyTimeout = time.Second
|
||||
var defaultEventuallyPollingInterval = 10 * time.Millisecond
|
||||
var defaultConsistentlyDuration = 100 * time.Millisecond
|
||||
var defaultConsistentlyPollingInterval = 10 * time.Millisecond
|
||||
|
||||
// RegisterFailHandler connects Ginkgo to Gomega. When a matcher fails
|
||||
// the fail handler passed into RegisterFailHandler is called.
|
||||
func RegisterFailHandler(handler types.GomegaFailHandler) {
|
||||
RegisterFailHandlerWithT(testingtsupport.EmptyTWithHelper{}, handler)
|
||||
}
|
||||
|
||||
// RegisterFailHandlerWithT ensures that the given types.TWithHelper and fail handler
|
||||
// are used globally.
|
||||
func RegisterFailHandlerWithT(t types.TWithHelper, handler types.GomegaFailHandler) {
|
||||
if handler == nil {
|
||||
globalFailWrapper = nil
|
||||
return
|
||||
}
|
||||
|
||||
globalFailWrapper = &types.GomegaFailWrapper{
|
||||
Fail: handler,
|
||||
TWithHelper: t,
|
||||
}
|
||||
}
|
||||
|
||||
// RegisterTestingT connects Gomega to Golang's XUnit style
|
||||
// Testing.T tests. It is now deprecated and you should use NewWithT() instead.
|
||||
//
|
||||
// Legacy Documentation:
|
||||
//
|
||||
// You'll need to call this at the top of each XUnit style test:
|
||||
//
|
||||
// func TestFarmHasCow(t *testing.T) {
|
||||
// RegisterTestingT(t)
|
||||
//
|
||||
// f := farm.New([]string{"Cow", "Horse"})
|
||||
// Expect(f.HasCow()).To(BeTrue(), "Farm should have cow")
|
||||
// }
|
||||
//
|
||||
// Note that this *testing.T is registered *globally* by Gomega (this is why you don't have to
|
||||
// pass `t` down to the matcher itself). This means that you cannot run the XUnit style tests
|
||||
// in parallel as the global fail handler cannot point to more than one testing.T at a time.
|
||||
//
|
||||
// NewWithT() does not have this limitation
|
||||
//
|
||||
// (As an aside: Ginkgo gets around this limitation by running parallel tests in different *processes*).
|
||||
func RegisterTestingT(t types.GomegaTestingT) {
|
||||
tWithHelper, hasHelper := t.(types.TWithHelper)
|
||||
if !hasHelper {
|
||||
RegisterFailHandler(testingtsupport.BuildTestingTGomegaFailWrapper(t).Fail)
|
||||
return
|
||||
}
|
||||
RegisterFailHandlerWithT(tWithHelper, testingtsupport.BuildTestingTGomegaFailWrapper(t).Fail)
|
||||
}
|
||||
|
||||
// InterceptGomegaFailures runs a given callback and returns an array of
|
||||
// failure messages generated by any Gomega assertions within the callback.
|
||||
//
|
||||
// This is accomplished by temporarily replacing the *global* fail handler
|
||||
// with a fail handler that simply annotates failures. The original fail handler
|
||||
// is reset when InterceptGomegaFailures returns.
|
||||
//
|
||||
// This is most useful when testing custom matchers, but can also be used to check
|
||||
// on a value using a Gomega assertion without causing a test failure.
|
||||
func InterceptGomegaFailures(f func()) []string {
|
||||
originalHandler := globalFailWrapper.Fail
|
||||
failures := []string{}
|
||||
RegisterFailHandler(func(message string, callerSkip ...int) {
|
||||
failures = append(failures, message)
|
||||
})
|
||||
f()
|
||||
RegisterFailHandler(originalHandler)
|
||||
return failures
|
||||
}
|
||||
|
||||
// Ω wraps an actual value allowing assertions to be made on it:
|
||||
// Ω("foo").Should(Equal("foo"))
|
||||
//
|
||||
// If Ω is passed more than one argument it will pass the *first* argument to the matcher.
|
||||
// All subsequent arguments will be required to be nil/zero.
|
||||
//
|
||||
// This is convenient if you want to make an assertion on a method/function that returns
|
||||
// a value and an error - a common patter in Go.
|
||||
//
|
||||
// For example, given a function with signature:
|
||||
// func MyAmazingThing() (int, error)
|
||||
//
|
||||
// Then:
|
||||
// Ω(MyAmazingThing()).Should(Equal(3))
|
||||
// Will succeed only if `MyAmazingThing()` returns `(3, nil)`
|
||||
//
|
||||
// Ω and Expect are identical
|
||||
func Ω(actual interface{}, extra ...interface{}) Assertion {
|
||||
return ExpectWithOffset(0, actual, extra...)
|
||||
}
|
||||
|
||||
// Expect wraps an actual value allowing assertions to be made on it:
|
||||
// Expect("foo").To(Equal("foo"))
|
||||
//
|
||||
// If Expect is passed more than one argument it will pass the *first* argument to the matcher.
|
||||
// All subsequent arguments will be required to be nil/zero.
|
||||
//
|
||||
// This is convenient if you want to make an assertion on a method/function that returns
|
||||
// a value and an error - a common patter in Go.
|
||||
//
|
||||
// For example, given a function with signature:
|
||||
// func MyAmazingThing() (int, error)
|
||||
//
|
||||
// Then:
|
||||
// Expect(MyAmazingThing()).Should(Equal(3))
|
||||
// Will succeed only if `MyAmazingThing()` returns `(3, nil)`
|
||||
//
|
||||
// Expect and Ω are identical
|
||||
func Expect(actual interface{}, extra ...interface{}) Assertion {
|
||||
return ExpectWithOffset(0, actual, extra...)
|
||||
}
|
||||
|
||||
// ExpectWithOffset wraps an actual value allowing assertions to be made on it:
|
||||
// ExpectWithOffset(1, "foo").To(Equal("foo"))
|
||||
//
|
||||
// Unlike `Expect` and `Ω`, `ExpectWithOffset` takes an additional integer argument
|
||||
// this is used to modify the call-stack offset when computing line numbers.
|
||||
//
|
||||
// This is most useful in helper functions that make assertions. If you want Gomega's
|
||||
// error message to refer to the calling line in the test (as opposed to the line in the helper function)
|
||||
// set the first argument of `ExpectWithOffset` appropriately.
|
||||
func ExpectWithOffset(offset int, actual interface{}, extra ...interface{}) Assertion {
|
||||
if globalFailWrapper == nil {
|
||||
panic(nilFailHandlerPanic)
|
||||
}
|
||||
return assertion.New(actual, globalFailWrapper, offset, extra...)
|
||||
}
|
||||
|
||||
// Eventually wraps an actual value allowing assertions to be made on it.
|
||||
// The assertion is tried periodically until it passes or a timeout occurs.
|
||||
//
|
||||
// Both the timeout and polling interval are configurable as optional arguments:
|
||||
// The first optional argument is the timeout
|
||||
// The second optional argument is the polling interval
|
||||
//
|
||||
// Both intervals can either be specified as time.Duration, parsable duration strings or as floats/integers. In the
|
||||
// last case they are interpreted as seconds.
|
||||
//
|
||||
// If Eventually is passed an actual that is a function taking no arguments and returning at least one value,
|
||||
// then Eventually will call the function periodically and try the matcher against the function's first return value.
|
||||
//
|
||||
// Example:
|
||||
//
|
||||
// Eventually(func() int {
|
||||
// return thingImPolling.Count()
|
||||
// }).Should(BeNumerically(">=", 17))
|
||||
//
|
||||
// Note that this example could be rewritten:
|
||||
//
|
||||
// Eventually(thingImPolling.Count).Should(BeNumerically(">=", 17))
|
||||
//
|
||||
// If the function returns more than one value, then Eventually will pass the first value to the matcher and
|
||||
// assert that all other values are nil/zero.
|
||||
// This allows you to pass Eventually a function that returns a value and an error - a common pattern in Go.
|
||||
//
|
||||
// For example, consider a method that returns a value and an error:
|
||||
// func FetchFromDB() (string, error)
|
||||
//
|
||||
// Then
|
||||
// Eventually(FetchFromDB).Should(Equal("hasselhoff"))
|
||||
//
|
||||
// Will pass only if the the returned error is nil and the returned string passes the matcher.
|
||||
//
|
||||
// Eventually's default timeout is 1 second, and its default polling interval is 10ms
|
||||
func Eventually(actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
return EventuallyWithOffset(0, actual, intervals...)
|
||||
}
|
||||
|
||||
// EventuallyWithOffset operates like Eventually but takes an additional
|
||||
// initial argument to indicate an offset in the call stack. This is useful when building helper
|
||||
// functions that contain matchers. To learn more, read about `ExpectWithOffset`.
|
||||
func EventuallyWithOffset(offset int, actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
if globalFailWrapper == nil {
|
||||
panic(nilFailHandlerPanic)
|
||||
}
|
||||
timeoutInterval := defaultEventuallyTimeout
|
||||
pollingInterval := defaultEventuallyPollingInterval
|
||||
if len(intervals) > 0 {
|
||||
timeoutInterval = toDuration(intervals[0])
|
||||
}
|
||||
if len(intervals) > 1 {
|
||||
pollingInterval = toDuration(intervals[1])
|
||||
}
|
||||
return asyncassertion.New(asyncassertion.AsyncAssertionTypeEventually, actual, globalFailWrapper, timeoutInterval, pollingInterval, offset)
|
||||
}
|
||||
|
||||
// Consistently wraps an actual value allowing assertions to be made on it.
|
||||
// The assertion is tried periodically and is required to pass for a period of time.
|
||||
//
|
||||
// Both the total time and polling interval are configurable as optional arguments:
|
||||
// The first optional argument is the duration that Consistently will run for
|
||||
// The second optional argument is the polling interval
|
||||
//
|
||||
// Both intervals can either be specified as time.Duration, parsable duration strings or as floats/integers. In the
|
||||
// last case they are interpreted as seconds.
|
||||
//
|
||||
// If Consistently is passed an actual that is a function taking no arguments and returning at least one value,
|
||||
// then Consistently will call the function periodically and try the matcher against the function's first return value.
|
||||
//
|
||||
// If the function returns more than one value, then Consistently will pass the first value to the matcher and
|
||||
// assert that all other values are nil/zero.
|
||||
// This allows you to pass Consistently a function that returns a value and an error - a common pattern in Go.
|
||||
//
|
||||
// Consistently is useful in cases where you want to assert that something *does not happen* over a period of tiem.
|
||||
// For example, you want to assert that a goroutine does *not* send data down a channel. In this case, you could:
|
||||
//
|
||||
// Consistently(channel).ShouldNot(Receive())
|
||||
//
|
||||
// Consistently's default duration is 100ms, and its default polling interval is 10ms
|
||||
func Consistently(actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
return ConsistentlyWithOffset(0, actual, intervals...)
|
||||
}
|
||||
|
||||
// ConsistentlyWithOffset operates like Consistnetly but takes an additional
|
||||
// initial argument to indicate an offset in the call stack. This is useful when building helper
|
||||
// functions that contain matchers. To learn more, read about `ExpectWithOffset`.
|
||||
func ConsistentlyWithOffset(offset int, actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
if globalFailWrapper == nil {
|
||||
panic(nilFailHandlerPanic)
|
||||
}
|
||||
timeoutInterval := defaultConsistentlyDuration
|
||||
pollingInterval := defaultConsistentlyPollingInterval
|
||||
if len(intervals) > 0 {
|
||||
timeoutInterval = toDuration(intervals[0])
|
||||
}
|
||||
if len(intervals) > 1 {
|
||||
pollingInterval = toDuration(intervals[1])
|
||||
}
|
||||
return asyncassertion.New(asyncassertion.AsyncAssertionTypeConsistently, actual, globalFailWrapper, timeoutInterval, pollingInterval, offset)
|
||||
}
|
||||
|
||||
// SetDefaultEventuallyTimeout sets the default timeout duration for Eventually. Eventually will repeatedly poll your condition until it succeeds, or until this timeout elapses.
|
||||
func SetDefaultEventuallyTimeout(t time.Duration) {
|
||||
defaultEventuallyTimeout = t
|
||||
}
|
||||
|
||||
// SetDefaultEventuallyPollingInterval sets the default polling interval for Eventually.
|
||||
func SetDefaultEventuallyPollingInterval(t time.Duration) {
|
||||
defaultEventuallyPollingInterval = t
|
||||
}
|
||||
|
||||
// SetDefaultConsistentlyDuration sets the default duration for Consistently. Consistently will verify that your condition is satsified for this long.
|
||||
func SetDefaultConsistentlyDuration(t time.Duration) {
|
||||
defaultConsistentlyDuration = t
|
||||
}
|
||||
|
||||
// SetDefaultConsistentlyPollingInterval sets the default polling interval for Consistently.
|
||||
func SetDefaultConsistentlyPollingInterval(t time.Duration) {
|
||||
defaultConsistentlyPollingInterval = t
|
||||
}
|
||||
|
||||
// AsyncAssertion is returned by Eventually and Consistently and polls the actual value passed into Eventually against
|
||||
// the matcher passed to the Should and ShouldNot methods.
|
||||
//
|
||||
// Both Should and ShouldNot take a variadic optionalDescription argument. This is passed on to
|
||||
// fmt.Sprintf() and is used to annotate failure messages. This allows you to make your failure messages more
|
||||
// descriptive.
|
||||
//
|
||||
// Both Should and ShouldNot return a boolean that is true if the assertion passed and false if it failed.
|
||||
//
|
||||
// Example:
|
||||
//
|
||||
// Eventually(myChannel).Should(Receive(), "Something should have come down the pipe.")
|
||||
// Consistently(myChannel).ShouldNot(Receive(), "Nothing should have come down the pipe.")
|
||||
type AsyncAssertion interface {
|
||||
Should(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
ShouldNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
}
|
||||
|
||||
// GomegaAsyncAssertion is deprecated in favor of AsyncAssertion, which does not stutter.
|
||||
type GomegaAsyncAssertion = AsyncAssertion
|
||||
|
||||
// Assertion is returned by Ω and Expect and compares the actual value to the matcher
|
||||
// passed to the Should/ShouldNot and To/ToNot/NotTo methods.
|
||||
//
|
||||
// Typically Should/ShouldNot are used with Ω and To/ToNot/NotTo are used with Expect
|
||||
// though this is not enforced.
|
||||
//
|
||||
// All methods take a variadic optionalDescription argument. This is passed on to fmt.Sprintf()
|
||||
// and is used to annotate failure messages.
|
||||
//
|
||||
// All methods return a bool that is true if hte assertion passed and false if it failed.
|
||||
//
|
||||
// Example:
|
||||
//
|
||||
// Ω(farm.HasCow()).Should(BeTrue(), "Farm %v should have a cow", farm)
|
||||
type Assertion interface {
|
||||
Should(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
ShouldNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
|
||||
To(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
ToNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
NotTo(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool
|
||||
}
|
||||
|
||||
// GomegaAssertion is deprecated in favor of Assertion, which does not stutter.
|
||||
type GomegaAssertion = Assertion
|
||||
|
||||
// OmegaMatcher is deprecated in favor of the better-named and better-organized types.GomegaMatcher but sticks around to support existing code that uses it
|
||||
type OmegaMatcher types.GomegaMatcher
|
||||
|
||||
// WithT wraps a *testing.T and provides `Expect`, `Eventually`, and `Consistently` methods. This allows you to leverage
|
||||
// Gomega's rich ecosystem of matchers in standard `testing` test suites.
|
||||
//
|
||||
// Use `NewWithT` to instantiate a `WithT`
|
||||
type WithT struct {
|
||||
t types.GomegaTestingT
|
||||
}
|
||||
|
||||
// GomegaWithT is deprecated in favor of gomega.WithT, which does not stutter.
|
||||
type GomegaWithT = WithT
|
||||
|
||||
// NewWithT takes a *testing.T and returngs a `gomega.WithT` allowing you to use `Expect`, `Eventually`, and `Consistently` along with
|
||||
// Gomega's rich ecosystem of matchers in standard `testing` test suits.
|
||||
//
|
||||
// func TestFarmHasCow(t *testing.T) {
|
||||
// g := gomega.NewWithT(t)
|
||||
//
|
||||
// f := farm.New([]string{"Cow", "Horse"})
|
||||
// g.Expect(f.HasCow()).To(BeTrue(), "Farm should have cow")
|
||||
// }
|
||||
func NewWithT(t types.GomegaTestingT) *WithT {
|
||||
return &WithT{
|
||||
t: t,
|
||||
}
|
||||
}
|
||||
|
||||
// NewGomegaWithT is deprecated in favor of gomega.NewWithT, which does not stutter.
|
||||
func NewGomegaWithT(t types.GomegaTestingT) *GomegaWithT {
|
||||
return NewWithT(t)
|
||||
}
|
||||
|
||||
// Expect is used to make assertions. See documentation for Expect.
|
||||
func (g *WithT) Expect(actual interface{}, extra ...interface{}) Assertion {
|
||||
return assertion.New(actual, testingtsupport.BuildTestingTGomegaFailWrapper(g.t), 0, extra...)
|
||||
}
|
||||
|
||||
// Eventually is used to make asynchronous assertions. See documentation for Eventually.
|
||||
func (g *WithT) Eventually(actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
timeoutInterval := defaultEventuallyTimeout
|
||||
pollingInterval := defaultEventuallyPollingInterval
|
||||
if len(intervals) > 0 {
|
||||
timeoutInterval = toDuration(intervals[0])
|
||||
}
|
||||
if len(intervals) > 1 {
|
||||
pollingInterval = toDuration(intervals[1])
|
||||
}
|
||||
return asyncassertion.New(asyncassertion.AsyncAssertionTypeEventually, actual, testingtsupport.BuildTestingTGomegaFailWrapper(g.t), timeoutInterval, pollingInterval, 0)
|
||||
}
|
||||
|
||||
// Consistently is used to make asynchronous assertions. See documentation for Consistently.
|
||||
func (g *WithT) Consistently(actual interface{}, intervals ...interface{}) AsyncAssertion {
|
||||
timeoutInterval := defaultConsistentlyDuration
|
||||
pollingInterval := defaultConsistentlyPollingInterval
|
||||
if len(intervals) > 0 {
|
||||
timeoutInterval = toDuration(intervals[0])
|
||||
}
|
||||
if len(intervals) > 1 {
|
||||
pollingInterval = toDuration(intervals[1])
|
||||
}
|
||||
return asyncassertion.New(asyncassertion.AsyncAssertionTypeConsistently, actual, testingtsupport.BuildTestingTGomegaFailWrapper(g.t), timeoutInterval, pollingInterval, 0)
|
||||
}
|
||||
|
||||
func toDuration(input interface{}) time.Duration {
|
||||
duration, ok := input.(time.Duration)
|
||||
if ok {
|
||||
return duration
|
||||
}
|
||||
|
||||
value := reflect.ValueOf(input)
|
||||
kind := reflect.TypeOf(input).Kind()
|
||||
|
||||
if reflect.Int <= kind && kind <= reflect.Int64 {
|
||||
return time.Duration(value.Int()) * time.Second
|
||||
} else if reflect.Uint <= kind && kind <= reflect.Uint64 {
|
||||
return time.Duration(value.Uint()) * time.Second
|
||||
} else if reflect.Float32 <= kind && kind <= reflect.Float64 {
|
||||
return time.Duration(value.Float() * float64(time.Second))
|
||||
} else if reflect.String == kind {
|
||||
duration, err := time.ParseDuration(value.String())
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("%#v is not a valid parsable duration string.", input))
|
||||
}
|
||||
return duration
|
||||
}
|
||||
|
||||
panic(fmt.Sprintf("%v is not a valid interval. Must be time.Duration, parsable duration string or a number.", input))
|
||||
}
|
105
vendor/github.com/onsi/gomega/internal/assertion/assertion.go
generated
vendored
Normal file
105
vendor/github.com/onsi/gomega/internal/assertion/assertion.go
generated
vendored
Normal file
@ -0,0 +1,105 @@
|
||||
package assertion
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type Assertion struct {
|
||||
actualInput interface{}
|
||||
failWrapper *types.GomegaFailWrapper
|
||||
offset int
|
||||
extra []interface{}
|
||||
}
|
||||
|
||||
func New(actualInput interface{}, failWrapper *types.GomegaFailWrapper, offset int, extra ...interface{}) *Assertion {
|
||||
return &Assertion{
|
||||
actualInput: actualInput,
|
||||
failWrapper: failWrapper,
|
||||
offset: offset,
|
||||
extra: extra,
|
||||
}
|
||||
}
|
||||
|
||||
func (assertion *Assertion) Should(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.vetExtras(optionalDescription...) && assertion.match(matcher, true, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *Assertion) ShouldNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.vetExtras(optionalDescription...) && assertion.match(matcher, false, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *Assertion) To(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.vetExtras(optionalDescription...) && assertion.match(matcher, true, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *Assertion) ToNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.vetExtras(optionalDescription...) && assertion.match(matcher, false, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *Assertion) NotTo(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.vetExtras(optionalDescription...) && assertion.match(matcher, false, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *Assertion) buildDescription(optionalDescription ...interface{}) string {
|
||||
switch len(optionalDescription) {
|
||||
case 0:
|
||||
return ""
|
||||
default:
|
||||
return fmt.Sprintf(optionalDescription[0].(string), optionalDescription[1:]...) + "\n"
|
||||
}
|
||||
}
|
||||
|
||||
func (assertion *Assertion) match(matcher types.GomegaMatcher, desiredMatch bool, optionalDescription ...interface{}) bool {
|
||||
matches, err := matcher.Match(assertion.actualInput)
|
||||
description := assertion.buildDescription(optionalDescription...)
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
if err != nil {
|
||||
assertion.failWrapper.Fail(description+err.Error(), 2+assertion.offset)
|
||||
return false
|
||||
}
|
||||
if matches != desiredMatch {
|
||||
var message string
|
||||
if desiredMatch {
|
||||
message = matcher.FailureMessage(assertion.actualInput)
|
||||
} else {
|
||||
message = matcher.NegatedFailureMessage(assertion.actualInput)
|
||||
}
|
||||
assertion.failWrapper.Fail(description+message, 2+assertion.offset)
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (assertion *Assertion) vetExtras(optionalDescription ...interface{}) bool {
|
||||
success, message := vetExtras(assertion.extra)
|
||||
if success {
|
||||
return true
|
||||
}
|
||||
|
||||
description := assertion.buildDescription(optionalDescription...)
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
assertion.failWrapper.Fail(description+message, 2+assertion.offset)
|
||||
return false
|
||||
}
|
||||
|
||||
func vetExtras(extras []interface{}) (bool, string) {
|
||||
for i, extra := range extras {
|
||||
if extra != nil {
|
||||
zeroValue := reflect.Zero(reflect.TypeOf(extra)).Interface()
|
||||
if !reflect.DeepEqual(zeroValue, extra) {
|
||||
message := fmt.Sprintf("Unexpected non-nil/non-zero extra argument at index %d:\n\t<%T>: %#v", i+1, extra, extra)
|
||||
return false, message
|
||||
}
|
||||
}
|
||||
}
|
||||
return true, ""
|
||||
}
|
194
vendor/github.com/onsi/gomega/internal/asyncassertion/async_assertion.go
generated
vendored
Normal file
194
vendor/github.com/onsi/gomega/internal/asyncassertion/async_assertion.go
generated
vendored
Normal file
@ -0,0 +1,194 @@
|
||||
package asyncassertion
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/onsi/gomega/internal/oraclematcher"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type AsyncAssertionType uint
|
||||
|
||||
const (
|
||||
AsyncAssertionTypeEventually AsyncAssertionType = iota
|
||||
AsyncAssertionTypeConsistently
|
||||
)
|
||||
|
||||
type AsyncAssertion struct {
|
||||
asyncType AsyncAssertionType
|
||||
actualInput interface{}
|
||||
timeoutInterval time.Duration
|
||||
pollingInterval time.Duration
|
||||
failWrapper *types.GomegaFailWrapper
|
||||
offset int
|
||||
}
|
||||
|
||||
func New(asyncType AsyncAssertionType, actualInput interface{}, failWrapper *types.GomegaFailWrapper, timeoutInterval time.Duration, pollingInterval time.Duration, offset int) *AsyncAssertion {
|
||||
actualType := reflect.TypeOf(actualInput)
|
||||
if actualType.Kind() == reflect.Func {
|
||||
if actualType.NumIn() != 0 || actualType.NumOut() == 0 {
|
||||
panic("Expected a function with no arguments and one or more return values.")
|
||||
}
|
||||
}
|
||||
|
||||
return &AsyncAssertion{
|
||||
asyncType: asyncType,
|
||||
actualInput: actualInput,
|
||||
failWrapper: failWrapper,
|
||||
timeoutInterval: timeoutInterval,
|
||||
pollingInterval: pollingInterval,
|
||||
offset: offset,
|
||||
}
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) Should(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.match(matcher, true, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) ShouldNot(matcher types.GomegaMatcher, optionalDescription ...interface{}) bool {
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
return assertion.match(matcher, false, optionalDescription...)
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) buildDescription(optionalDescription ...interface{}) string {
|
||||
switch len(optionalDescription) {
|
||||
case 0:
|
||||
return ""
|
||||
default:
|
||||
return fmt.Sprintf(optionalDescription[0].(string), optionalDescription[1:]...) + "\n"
|
||||
}
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) actualInputIsAFunction() bool {
|
||||
actualType := reflect.TypeOf(assertion.actualInput)
|
||||
return actualType.Kind() == reflect.Func && actualType.NumIn() == 0 && actualType.NumOut() > 0
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) pollActual() (interface{}, error) {
|
||||
if assertion.actualInputIsAFunction() {
|
||||
values := reflect.ValueOf(assertion.actualInput).Call([]reflect.Value{})
|
||||
|
||||
extras := []interface{}{}
|
||||
for _, value := range values[1:] {
|
||||
extras = append(extras, value.Interface())
|
||||
}
|
||||
|
||||
success, message := vetExtras(extras)
|
||||
|
||||
if !success {
|
||||
return nil, errors.New(message)
|
||||
}
|
||||
|
||||
return values[0].Interface(), nil
|
||||
}
|
||||
|
||||
return assertion.actualInput, nil
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) matcherMayChange(matcher types.GomegaMatcher, value interface{}) bool {
|
||||
if assertion.actualInputIsAFunction() {
|
||||
return true
|
||||
}
|
||||
|
||||
return oraclematcher.MatchMayChangeInTheFuture(matcher, value)
|
||||
}
|
||||
|
||||
func (assertion *AsyncAssertion) match(matcher types.GomegaMatcher, desiredMatch bool, optionalDescription ...interface{}) bool {
|
||||
timer := time.Now()
|
||||
timeout := time.After(assertion.timeoutInterval)
|
||||
|
||||
description := assertion.buildDescription(optionalDescription...)
|
||||
|
||||
var matches bool
|
||||
var err error
|
||||
mayChange := true
|
||||
value, err := assertion.pollActual()
|
||||
if err == nil {
|
||||
mayChange = assertion.matcherMayChange(matcher, value)
|
||||
matches, err = matcher.Match(value)
|
||||
}
|
||||
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
|
||||
fail := func(preamble string) {
|
||||
errMsg := ""
|
||||
message := ""
|
||||
if err != nil {
|
||||
errMsg = "Error: " + err.Error()
|
||||
} else {
|
||||
if desiredMatch {
|
||||
message = matcher.FailureMessage(value)
|
||||
} else {
|
||||
message = matcher.NegatedFailureMessage(value)
|
||||
}
|
||||
}
|
||||
assertion.failWrapper.TWithHelper.Helper()
|
||||
assertion.failWrapper.Fail(fmt.Sprintf("%s after %.3fs.\n%s%s%s", preamble, time.Since(timer).Seconds(), description, message, errMsg), 3+assertion.offset)
|
||||
}
|
||||
|
||||
if assertion.asyncType == AsyncAssertionTypeEventually {
|
||||
for {
|
||||
if err == nil && matches == desiredMatch {
|
||||
return true
|
||||
}
|
||||
|
||||
if !mayChange {
|
||||
fail("No future change is possible. Bailing out early")
|
||||
return false
|
||||
}
|
||||
|
||||
select {
|
||||
case <-time.After(assertion.pollingInterval):
|
||||
value, err = assertion.pollActual()
|
||||
if err == nil {
|
||||
mayChange = assertion.matcherMayChange(matcher, value)
|
||||
matches, err = matcher.Match(value)
|
||||
}
|
||||
case <-timeout:
|
||||
fail("Timed out")
|
||||
return false
|
||||
}
|
||||
}
|
||||
} else if assertion.asyncType == AsyncAssertionTypeConsistently {
|
||||
for {
|
||||
if !(err == nil && matches == desiredMatch) {
|
||||
fail("Failed")
|
||||
return false
|
||||
}
|
||||
|
||||
if !mayChange {
|
||||
return true
|
||||
}
|
||||
|
||||
select {
|
||||
case <-time.After(assertion.pollingInterval):
|
||||
value, err = assertion.pollActual()
|
||||
if err == nil {
|
||||
mayChange = assertion.matcherMayChange(matcher, value)
|
||||
matches, err = matcher.Match(value)
|
||||
}
|
||||
case <-timeout:
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func vetExtras(extras []interface{}) (bool, string) {
|
||||
for i, extra := range extras {
|
||||
if extra != nil {
|
||||
zeroValue := reflect.Zero(reflect.TypeOf(extra)).Interface()
|
||||
if !reflect.DeepEqual(zeroValue, extra) {
|
||||
message := fmt.Sprintf("Unexpected non-nil/non-zero extra argument at index %d:\n\t<%T>: %#v", i+1, extra, extra)
|
||||
return false, message
|
||||
}
|
||||
}
|
||||
}
|
||||
return true, ""
|
||||
}
|
25
vendor/github.com/onsi/gomega/internal/oraclematcher/oracle_matcher.go
generated
vendored
Normal file
25
vendor/github.com/onsi/gomega/internal/oraclematcher/oracle_matcher.go
generated
vendored
Normal file
@ -0,0 +1,25 @@
|
||||
package oraclematcher
|
||||
|
||||
import "github.com/onsi/gomega/types"
|
||||
|
||||
/*
|
||||
GomegaMatchers that also match the OracleMatcher interface can convey information about
|
||||
whether or not their result will change upon future attempts.
|
||||
|
||||
This allows `Eventually` and `Consistently` to short circuit if success becomes impossible.
|
||||
|
||||
For example, a process' exit code can never change. So, gexec's Exit matcher returns `true`
|
||||
for `MatchMayChangeInTheFuture` until the process exits, at which point it returns `false` forevermore.
|
||||
*/
|
||||
type OracleMatcher interface {
|
||||
MatchMayChangeInTheFuture(actual interface{}) bool
|
||||
}
|
||||
|
||||
func MatchMayChangeInTheFuture(matcher types.GomegaMatcher, value interface{}) bool {
|
||||
oracleMatcher, ok := matcher.(OracleMatcher)
|
||||
if !ok {
|
||||
return true
|
||||
}
|
||||
|
||||
return oracleMatcher.MatchMayChangeInTheFuture(value)
|
||||
}
|
60
vendor/github.com/onsi/gomega/internal/testingtsupport/testing_t_support.go
generated
vendored
Normal file
60
vendor/github.com/onsi/gomega/internal/testingtsupport/testing_t_support.go
generated
vendored
Normal file
@ -0,0 +1,60 @@
|
||||
package testingtsupport
|
||||
|
||||
import (
|
||||
"regexp"
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
var StackTracePruneRE = regexp.MustCompile(`\/gomega\/|\/ginkgo\/|\/pkg\/testing\/|\/pkg\/runtime\/`)
|
||||
|
||||
type EmptyTWithHelper struct{}
|
||||
|
||||
func (e EmptyTWithHelper) Helper() {}
|
||||
|
||||
type gomegaTestingT interface {
|
||||
Fatalf(format string, args ...interface{})
|
||||
}
|
||||
|
||||
func BuildTestingTGomegaFailWrapper(t gomegaTestingT) *types.GomegaFailWrapper {
|
||||
tWithHelper, hasHelper := t.(types.TWithHelper)
|
||||
if !hasHelper {
|
||||
tWithHelper = EmptyTWithHelper{}
|
||||
}
|
||||
|
||||
fail := func(message string, callerSkip ...int) {
|
||||
if hasHelper {
|
||||
tWithHelper.Helper()
|
||||
t.Fatalf("\n%s", message)
|
||||
} else {
|
||||
skip := 2
|
||||
if len(callerSkip) > 0 {
|
||||
skip += callerSkip[0]
|
||||
}
|
||||
stackTrace := pruneStack(string(debug.Stack()), skip)
|
||||
t.Fatalf("\n%s\n%s\n", stackTrace, message)
|
||||
}
|
||||
}
|
||||
|
||||
return &types.GomegaFailWrapper{
|
||||
Fail: fail,
|
||||
TWithHelper: tWithHelper,
|
||||
}
|
||||
}
|
||||
|
||||
func pruneStack(fullStackTrace string, skip int) string {
|
||||
stack := strings.Split(fullStackTrace, "\n")[1:]
|
||||
if len(stack) > 2*skip {
|
||||
stack = stack[2*skip:]
|
||||
}
|
||||
prunedStack := []string{}
|
||||
for i := 0; i < len(stack)/2; i++ {
|
||||
if !StackTracePruneRE.Match([]byte(stack[i*2])) {
|
||||
prunedStack = append(prunedStack, stack[i*2])
|
||||
prunedStack = append(prunedStack, stack[i*2+1])
|
||||
}
|
||||
}
|
||||
return strings.Join(prunedStack, "\n")
|
||||
}
|
427
vendor/github.com/onsi/gomega/matchers.go
generated
vendored
Normal file
427
vendor/github.com/onsi/gomega/matchers.go
generated
vendored
Normal file
@ -0,0 +1,427 @@
|
||||
package gomega
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/onsi/gomega/matchers"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
//Equal uses reflect.DeepEqual to compare actual with expected. Equal is strict about
|
||||
//types when performing comparisons.
|
||||
//It is an error for both actual and expected to be nil. Use BeNil() instead.
|
||||
func Equal(expected interface{}) types.GomegaMatcher {
|
||||
return &matchers.EqualMatcher{
|
||||
Expected: expected,
|
||||
}
|
||||
}
|
||||
|
||||
//BeEquivalentTo is more lax than Equal, allowing equality between different types.
|
||||
//This is done by converting actual to have the type of expected before
|
||||
//attempting equality with reflect.DeepEqual.
|
||||
//It is an error for actual and expected to be nil. Use BeNil() instead.
|
||||
func BeEquivalentTo(expected interface{}) types.GomegaMatcher {
|
||||
return &matchers.BeEquivalentToMatcher{
|
||||
Expected: expected,
|
||||
}
|
||||
}
|
||||
|
||||
//BeIdenticalTo uses the == operator to compare actual with expected.
|
||||
//BeIdenticalTo is strict about types when performing comparisons.
|
||||
//It is an error for both actual and expected to be nil. Use BeNil() instead.
|
||||
func BeIdenticalTo(expected interface{}) types.GomegaMatcher {
|
||||
return &matchers.BeIdenticalToMatcher{
|
||||
Expected: expected,
|
||||
}
|
||||
}
|
||||
|
||||
//BeNil succeeds if actual is nil
|
||||
func BeNil() types.GomegaMatcher {
|
||||
return &matchers.BeNilMatcher{}
|
||||
}
|
||||
|
||||
//BeTrue succeeds if actual is true
|
||||
func BeTrue() types.GomegaMatcher {
|
||||
return &matchers.BeTrueMatcher{}
|
||||
}
|
||||
|
||||
//BeFalse succeeds if actual is false
|
||||
func BeFalse() types.GomegaMatcher {
|
||||
return &matchers.BeFalseMatcher{}
|
||||
}
|
||||
|
||||
//HaveOccurred succeeds if actual is a non-nil error
|
||||
//The typical Go error checking pattern looks like:
|
||||
// err := SomethingThatMightFail()
|
||||
// Expect(err).ShouldNot(HaveOccurred())
|
||||
func HaveOccurred() types.GomegaMatcher {
|
||||
return &matchers.HaveOccurredMatcher{}
|
||||
}
|
||||
|
||||
//Succeed passes if actual is a nil error
|
||||
//Succeed is intended to be used with functions that return a single error value. Instead of
|
||||
// err := SomethingThatMightFail()
|
||||
// Expect(err).ShouldNot(HaveOccurred())
|
||||
//
|
||||
//You can write:
|
||||
// Expect(SomethingThatMightFail()).Should(Succeed())
|
||||
//
|
||||
//It is a mistake to use Succeed with a function that has multiple return values. Gomega's Ω and Expect
|
||||
//functions automatically trigger failure if any return values after the first return value are non-zero/non-nil.
|
||||
//This means that Ω(MultiReturnFunc()).ShouldNot(Succeed()) can never pass.
|
||||
func Succeed() types.GomegaMatcher {
|
||||
return &matchers.SucceedMatcher{}
|
||||
}
|
||||
|
||||
//MatchError succeeds if actual is a non-nil error that matches the passed in string/error.
|
||||
//
|
||||
//These are valid use-cases:
|
||||
// Expect(err).Should(MatchError("an error")) //asserts that err.Error() == "an error"
|
||||
// Expect(err).Should(MatchError(SomeError)) //asserts that err == SomeError (via reflect.DeepEqual)
|
||||
//
|
||||
//It is an error for err to be nil or an object that does not implement the Error interface
|
||||
func MatchError(expected interface{}) types.GomegaMatcher {
|
||||
return &matchers.MatchErrorMatcher{
|
||||
Expected: expected,
|
||||
}
|
||||
}
|
||||
|
||||
//BeClosed succeeds if actual is a closed channel.
|
||||
//It is an error to pass a non-channel to BeClosed, it is also an error to pass nil
|
||||
//
|
||||
//In order to check whether or not the channel is closed, Gomega must try to read from the channel
|
||||
//(even in the `ShouldNot(BeClosed())` case). You should keep this in mind if you wish to make subsequent assertions about
|
||||
//values coming down the channel.
|
||||
//
|
||||
//Also, if you are testing that a *buffered* channel is closed you must first read all values out of the channel before
|
||||
//asserting that it is closed (it is not possible to detect that a buffered-channel has been closed until all its buffered values are read).
|
||||
//
|
||||
//Finally, as a corollary: it is an error to check whether or not a send-only channel is closed.
|
||||
func BeClosed() types.GomegaMatcher {
|
||||
return &matchers.BeClosedMatcher{}
|
||||
}
|
||||
|
||||
//Receive succeeds if there is a value to be received on actual.
|
||||
//Actual must be a channel (and cannot be a send-only channel) -- anything else is an error.
|
||||
//
|
||||
//Receive returns immediately and never blocks:
|
||||
//
|
||||
//- If there is nothing on the channel `c` then Expect(c).Should(Receive()) will fail and Ω(c).ShouldNot(Receive()) will pass.
|
||||
//
|
||||
//- If the channel `c` is closed then Expect(c).Should(Receive()) will fail and Ω(c).ShouldNot(Receive()) will pass.
|
||||
//
|
||||
//- If there is something on the channel `c` ready to be read, then Expect(c).Should(Receive()) will pass and Ω(c).ShouldNot(Receive()) will fail.
|
||||
//
|
||||
//If you have a go-routine running in the background that will write to channel `c` you can:
|
||||
// Eventually(c).Should(Receive())
|
||||
//
|
||||
//This will timeout if nothing gets sent to `c` (you can modify the timeout interval as you normally do with `Eventually`)
|
||||
//
|
||||
//A similar use-case is to assert that no go-routine writes to a channel (for a period of time). You can do this with `Consistently`:
|
||||
// Consistently(c).ShouldNot(Receive())
|
||||
//
|
||||
//You can pass `Receive` a matcher. If you do so, it will match the received object against the matcher. For example:
|
||||
// Expect(c).Should(Receive(Equal("foo")))
|
||||
//
|
||||
//When given a matcher, `Receive` will always fail if there is nothing to be received on the channel.
|
||||
//
|
||||
//Passing Receive a matcher is especially useful when paired with Eventually:
|
||||
//
|
||||
// Eventually(c).Should(Receive(ContainSubstring("bar")))
|
||||
//
|
||||
//will repeatedly attempt to pull values out of `c` until a value matching "bar" is received.
|
||||
//
|
||||
//Finally, if you want to have a reference to the value *sent* to the channel you can pass the `Receive` matcher a pointer to a variable of the appropriate type:
|
||||
// var myThing thing
|
||||
// Eventually(thingChan).Should(Receive(&myThing))
|
||||
// Expect(myThing.Sprocket).Should(Equal("foo"))
|
||||
// Expect(myThing.IsValid()).Should(BeTrue())
|
||||
func Receive(args ...interface{}) types.GomegaMatcher {
|
||||
var arg interface{}
|
||||
if len(args) > 0 {
|
||||
arg = args[0]
|
||||
}
|
||||
|
||||
return &matchers.ReceiveMatcher{
|
||||
Arg: arg,
|
||||
}
|
||||
}
|
||||
|
||||
//BeSent succeeds if a value can be sent to actual.
|
||||
//Actual must be a channel (and cannot be a receive-only channel) that can sent the type of the value passed into BeSent -- anything else is an error.
|
||||
//In addition, actual must not be closed.
|
||||
//
|
||||
//BeSent never blocks:
|
||||
//
|
||||
//- If the channel `c` is not ready to receive then Expect(c).Should(BeSent("foo")) will fail immediately
|
||||
//- If the channel `c` is eventually ready to receive then Eventually(c).Should(BeSent("foo")) will succeed.. presuming the channel becomes ready to receive before Eventually's timeout
|
||||
//- If the channel `c` is closed then Expect(c).Should(BeSent("foo")) and Ω(c).ShouldNot(BeSent("foo")) will both fail immediately
|
||||
//
|
||||
//Of course, the value is actually sent to the channel. The point of `BeSent` is less to make an assertion about the availability of the channel (which is typically an implementation detail that your test should not be concerned with).
|
||||
//Rather, the point of `BeSent` is to make it possible to easily and expressively write tests that can timeout on blocked channel sends.
|
||||
func BeSent(arg interface{}) types.GomegaMatcher {
|
||||
return &matchers.BeSentMatcher{
|
||||
Arg: arg,
|
||||
}
|
||||
}
|
||||
|
||||
//MatchRegexp succeeds if actual is a string or stringer that matches the
|
||||
//passed-in regexp. Optional arguments can be provided to construct a regexp
|
||||
//via fmt.Sprintf().
|
||||
func MatchRegexp(regexp string, args ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.MatchRegexpMatcher{
|
||||
Regexp: regexp,
|
||||
Args: args,
|
||||
}
|
||||
}
|
||||
|
||||
//ContainSubstring succeeds if actual is a string or stringer that contains the
|
||||
//passed-in substring. Optional arguments can be provided to construct the substring
|
||||
//via fmt.Sprintf().
|
||||
func ContainSubstring(substr string, args ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.ContainSubstringMatcher{
|
||||
Substr: substr,
|
||||
Args: args,
|
||||
}
|
||||
}
|
||||
|
||||
//HavePrefix succeeds if actual is a string or stringer that contains the
|
||||
//passed-in string as a prefix. Optional arguments can be provided to construct
|
||||
//via fmt.Sprintf().
|
||||
func HavePrefix(prefix string, args ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.HavePrefixMatcher{
|
||||
Prefix: prefix,
|
||||
Args: args,
|
||||
}
|
||||
}
|
||||
|
||||
//HaveSuffix succeeds if actual is a string or stringer that contains the
|
||||
//passed-in string as a suffix. Optional arguments can be provided to construct
|
||||
//via fmt.Sprintf().
|
||||
func HaveSuffix(suffix string, args ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.HaveSuffixMatcher{
|
||||
Suffix: suffix,
|
||||
Args: args,
|
||||
}
|
||||
}
|
||||
|
||||
//MatchJSON succeeds if actual is a string or stringer of JSON that matches
|
||||
//the expected JSON. The JSONs are decoded and the resulting objects are compared via
|
||||
//reflect.DeepEqual so things like key-ordering and whitespace shouldn't matter.
|
||||
func MatchJSON(json interface{}) types.GomegaMatcher {
|
||||
return &matchers.MatchJSONMatcher{
|
||||
JSONToMatch: json,
|
||||
}
|
||||
}
|
||||
|
||||
//MatchXML succeeds if actual is a string or stringer of XML that matches
|
||||
//the expected XML. The XMLs are decoded and the resulting objects are compared via
|
||||
//reflect.DeepEqual so things like whitespaces shouldn't matter.
|
||||
func MatchXML(xml interface{}) types.GomegaMatcher {
|
||||
return &matchers.MatchXMLMatcher{
|
||||
XMLToMatch: xml,
|
||||
}
|
||||
}
|
||||
|
||||
//MatchYAML succeeds if actual is a string or stringer of YAML that matches
|
||||
//the expected YAML. The YAML's are decoded and the resulting objects are compared via
|
||||
//reflect.DeepEqual so things like key-ordering and whitespace shouldn't matter.
|
||||
func MatchYAML(yaml interface{}) types.GomegaMatcher {
|
||||
return &matchers.MatchYAMLMatcher{
|
||||
YAMLToMatch: yaml,
|
||||
}
|
||||
}
|
||||
|
||||
//BeEmpty succeeds if actual is empty. Actual must be of type string, array, map, chan, or slice.
|
||||
func BeEmpty() types.GomegaMatcher {
|
||||
return &matchers.BeEmptyMatcher{}
|
||||
}
|
||||
|
||||
//HaveLen succeeds if actual has the passed-in length. Actual must be of type string, array, map, chan, or slice.
|
||||
func HaveLen(count int) types.GomegaMatcher {
|
||||
return &matchers.HaveLenMatcher{
|
||||
Count: count,
|
||||
}
|
||||
}
|
||||
|
||||
//HaveCap succeeds if actual has the passed-in capacity. Actual must be of type array, chan, or slice.
|
||||
func HaveCap(count int) types.GomegaMatcher {
|
||||
return &matchers.HaveCapMatcher{
|
||||
Count: count,
|
||||
}
|
||||
}
|
||||
|
||||
//BeZero succeeds if actual is the zero value for its type or if actual is nil.
|
||||
func BeZero() types.GomegaMatcher {
|
||||
return &matchers.BeZeroMatcher{}
|
||||
}
|
||||
|
||||
//ContainElement succeeds if actual contains the passed in element.
|
||||
//By default ContainElement() uses Equal() to perform the match, however a
|
||||
//matcher can be passed in instead:
|
||||
// Expect([]string{"Foo", "FooBar"}).Should(ContainElement(ContainSubstring("Bar")))
|
||||
//
|
||||
//Actual must be an array, slice or map.
|
||||
//For maps, ContainElement searches through the map's values.
|
||||
func ContainElement(element interface{}) types.GomegaMatcher {
|
||||
return &matchers.ContainElementMatcher{
|
||||
Element: element,
|
||||
}
|
||||
}
|
||||
|
||||
//ConsistOf succeeds if actual contains precisely the elements passed into the matcher. The ordering of the elements does not matter.
|
||||
//By default ConsistOf() uses Equal() to match the elements, however custom matchers can be passed in instead. Here are some examples:
|
||||
//
|
||||
// Expect([]string{"Foo", "FooBar"}).Should(ConsistOf("FooBar", "Foo"))
|
||||
// Expect([]string{"Foo", "FooBar"}).Should(ConsistOf(ContainSubstring("Bar"), "Foo"))
|
||||
// Expect([]string{"Foo", "FooBar"}).Should(ConsistOf(ContainSubstring("Foo"), ContainSubstring("Foo")))
|
||||
//
|
||||
//Actual must be an array, slice or map. For maps, ConsistOf matches against the map's values.
|
||||
//
|
||||
//You typically pass variadic arguments to ConsistOf (as in the examples above). However, if you need to pass in a slice you can provided that it
|
||||
//is the only element passed in to ConsistOf:
|
||||
//
|
||||
// Expect([]string{"Foo", "FooBar"}).Should(ConsistOf([]string{"FooBar", "Foo"}))
|
||||
//
|
||||
//Note that Go's type system does not allow you to write this as ConsistOf([]string{"FooBar", "Foo"}...) as []string and []interface{} are different types - hence the need for this special rule.
|
||||
func ConsistOf(elements ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.ConsistOfMatcher{
|
||||
Elements: elements,
|
||||
}
|
||||
}
|
||||
|
||||
//HaveKey succeeds if actual is a map with the passed in key.
|
||||
//By default HaveKey uses Equal() to perform the match, however a
|
||||
//matcher can be passed in instead:
|
||||
// Expect(map[string]string{"Foo": "Bar", "BazFoo": "Duck"}).Should(HaveKey(MatchRegexp(`.+Foo$`)))
|
||||
func HaveKey(key interface{}) types.GomegaMatcher {
|
||||
return &matchers.HaveKeyMatcher{
|
||||
Key: key,
|
||||
}
|
||||
}
|
||||
|
||||
//HaveKeyWithValue succeeds if actual is a map with the passed in key and value.
|
||||
//By default HaveKeyWithValue uses Equal() to perform the match, however a
|
||||
//matcher can be passed in instead:
|
||||
// Expect(map[string]string{"Foo": "Bar", "BazFoo": "Duck"}).Should(HaveKeyWithValue("Foo", "Bar"))
|
||||
// Expect(map[string]string{"Foo": "Bar", "BazFoo": "Duck"}).Should(HaveKeyWithValue(MatchRegexp(`.+Foo$`), "Bar"))
|
||||
func HaveKeyWithValue(key interface{}, value interface{}) types.GomegaMatcher {
|
||||
return &matchers.HaveKeyWithValueMatcher{
|
||||
Key: key,
|
||||
Value: value,
|
||||
}
|
||||
}
|
||||
|
||||
//BeNumerically performs numerical assertions in a type-agnostic way.
|
||||
//Actual and expected should be numbers, though the specific type of
|
||||
//number is irrelevant (float32, float64, uint8, etc...).
|
||||
//
|
||||
//There are six, self-explanatory, supported comparators:
|
||||
// Expect(1.0).Should(BeNumerically("==", 1))
|
||||
// Expect(1.0).Should(BeNumerically("~", 0.999, 0.01))
|
||||
// Expect(1.0).Should(BeNumerically(">", 0.9))
|
||||
// Expect(1.0).Should(BeNumerically(">=", 1.0))
|
||||
// Expect(1.0).Should(BeNumerically("<", 3))
|
||||
// Expect(1.0).Should(BeNumerically("<=", 1.0))
|
||||
func BeNumerically(comparator string, compareTo ...interface{}) types.GomegaMatcher {
|
||||
return &matchers.BeNumericallyMatcher{
|
||||
Comparator: comparator,
|
||||
CompareTo: compareTo,
|
||||
}
|
||||
}
|
||||
|
||||
//BeTemporally compares time.Time's like BeNumerically
|
||||
//Actual and expected must be time.Time. The comparators are the same as for BeNumerically
|
||||
// Expect(time.Now()).Should(BeTemporally(">", time.Time{}))
|
||||
// Expect(time.Now()).Should(BeTemporally("~", time.Now(), time.Second))
|
||||
func BeTemporally(comparator string, compareTo time.Time, threshold ...time.Duration) types.GomegaMatcher {
|
||||
return &matchers.BeTemporallyMatcher{
|
||||
Comparator: comparator,
|
||||
CompareTo: compareTo,
|
||||
Threshold: threshold,
|
||||
}
|
||||
}
|
||||
|
||||
//BeAssignableToTypeOf succeeds if actual is assignable to the type of expected.
|
||||
//It will return an error when one of the values is nil.
|
||||
// Expect(0).Should(BeAssignableToTypeOf(0)) // Same values
|
||||
// Expect(5).Should(BeAssignableToTypeOf(-1)) // different values same type
|
||||
// Expect("foo").Should(BeAssignableToTypeOf("bar")) // different values same type
|
||||
// Expect(struct{ Foo string }{}).Should(BeAssignableToTypeOf(struct{ Foo string }{}))
|
||||
func BeAssignableToTypeOf(expected interface{}) types.GomegaMatcher {
|
||||
return &matchers.AssignableToTypeOfMatcher{
|
||||
Expected: expected,
|
||||
}
|
||||
}
|
||||
|
||||
//Panic succeeds if actual is a function that, when invoked, panics.
|
||||
//Actual must be a function that takes no arguments and returns no results.
|
||||
func Panic() types.GomegaMatcher {
|
||||
return &matchers.PanicMatcher{}
|
||||
}
|
||||
|
||||
//BeAnExistingFile succeeds if a file exists.
|
||||
//Actual must be a string representing the abs path to the file being checked.
|
||||
func BeAnExistingFile() types.GomegaMatcher {
|
||||
return &matchers.BeAnExistingFileMatcher{}
|
||||
}
|
||||
|
||||
//BeARegularFile succeeds if a file exists and is a regular file.
|
||||
//Actual must be a string representing the abs path to the file being checked.
|
||||
func BeARegularFile() types.GomegaMatcher {
|
||||
return &matchers.BeARegularFileMatcher{}
|
||||
}
|
||||
|
||||
//BeADirectory succeeds if a file exists and is a directory.
|
||||
//Actual must be a string representing the abs path to the file being checked.
|
||||
func BeADirectory() types.GomegaMatcher {
|
||||
return &matchers.BeADirectoryMatcher{}
|
||||
}
|
||||
|
||||
//And succeeds only if all of the given matchers succeed.
|
||||
//The matchers are tried in order, and will fail-fast if one doesn't succeed.
|
||||
// Expect("hi").To(And(HaveLen(2), Equal("hi"))
|
||||
//
|
||||
//And(), Or(), Not() and WithTransform() allow matchers to be composed into complex expressions.
|
||||
func And(ms ...types.GomegaMatcher) types.GomegaMatcher {
|
||||
return &matchers.AndMatcher{Matchers: ms}
|
||||
}
|
||||
|
||||
//SatisfyAll is an alias for And().
|
||||
// Expect("hi").Should(SatisfyAll(HaveLen(2), Equal("hi")))
|
||||
func SatisfyAll(matchers ...types.GomegaMatcher) types.GomegaMatcher {
|
||||
return And(matchers...)
|
||||
}
|
||||
|
||||
//Or succeeds if any of the given matchers succeed.
|
||||
//The matchers are tried in order and will return immediately upon the first successful match.
|
||||
// Expect("hi").To(Or(HaveLen(3), HaveLen(2))
|
||||
//
|
||||
//And(), Or(), Not() and WithTransform() allow matchers to be composed into complex expressions.
|
||||
func Or(ms ...types.GomegaMatcher) types.GomegaMatcher {
|
||||
return &matchers.OrMatcher{Matchers: ms}
|
||||
}
|
||||
|
||||
//SatisfyAny is an alias for Or().
|
||||
// Expect("hi").SatisfyAny(Or(HaveLen(3), HaveLen(2))
|
||||
func SatisfyAny(matchers ...types.GomegaMatcher) types.GomegaMatcher {
|
||||
return Or(matchers...)
|
||||
}
|
||||
|
||||
//Not negates the given matcher; it succeeds if the given matcher fails.
|
||||
// Expect(1).To(Not(Equal(2))
|
||||
//
|
||||
//And(), Or(), Not() and WithTransform() allow matchers to be composed into complex expressions.
|
||||
func Not(matcher types.GomegaMatcher) types.GomegaMatcher {
|
||||
return &matchers.NotMatcher{Matcher: matcher}
|
||||
}
|
||||
|
||||
//WithTransform applies the `transform` to the actual value and matches it against `matcher`.
|
||||
//The given transform must be a function of one parameter that returns one value.
|
||||
// var plus1 = func(i int) int { return i + 1 }
|
||||
// Expect(1).To(WithTransform(plus1, Equal(2))
|
||||
//
|
||||
//And(), Or(), Not() and WithTransform() allow matchers to be composed into complex expressions.
|
||||
func WithTransform(transform interface{}, matcher types.GomegaMatcher) types.GomegaMatcher {
|
||||
return matchers.NewWithTransformMatcher(transform, matcher)
|
||||
}
|
63
vendor/github.com/onsi/gomega/matchers/and.go
generated
vendored
Normal file
63
vendor/github.com/onsi/gomega/matchers/and.go
generated
vendored
Normal file
@ -0,0 +1,63 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
"github.com/onsi/gomega/internal/oraclematcher"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type AndMatcher struct {
|
||||
Matchers []types.GomegaMatcher
|
||||
|
||||
// state
|
||||
firstFailedMatcher types.GomegaMatcher
|
||||
}
|
||||
|
||||
func (m *AndMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
m.firstFailedMatcher = nil
|
||||
for _, matcher := range m.Matchers {
|
||||
success, err := matcher.Match(actual)
|
||||
if !success || err != nil {
|
||||
m.firstFailedMatcher = matcher
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (m *AndMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return m.firstFailedMatcher.FailureMessage(actual)
|
||||
}
|
||||
|
||||
func (m *AndMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
// not the most beautiful list of matchers, but not bad either...
|
||||
return format.Message(actual, fmt.Sprintf("To not satisfy all of these matchers: %s", m.Matchers))
|
||||
}
|
||||
|
||||
func (m *AndMatcher) MatchMayChangeInTheFuture(actual interface{}) bool {
|
||||
/*
|
||||
Example with 3 matchers: A, B, C
|
||||
|
||||
Match evaluates them: T, F, <?> => F
|
||||
So match is currently F, what should MatchMayChangeInTheFuture() return?
|
||||
Seems like it only depends on B, since currently B MUST change to allow the result to become T
|
||||
|
||||
Match eval: T, T, T => T
|
||||
So match is currently T, what should MatchMayChangeInTheFuture() return?
|
||||
Seems to depend on ANY of them being able to change to F.
|
||||
*/
|
||||
|
||||
if m.firstFailedMatcher == nil {
|
||||
// so all matchers succeeded.. Any one of them changing would change the result.
|
||||
for _, matcher := range m.Matchers {
|
||||
if oraclematcher.MatchMayChangeInTheFuture(matcher, actual) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false // none of were going to change
|
||||
}
|
||||
// one of the matchers failed.. it must be able to change in order to affect the result
|
||||
return oraclematcher.MatchMayChangeInTheFuture(m.firstFailedMatcher, actual)
|
||||
}
|
35
vendor/github.com/onsi/gomega/matchers/assignable_to_type_of_matcher.go
generated
vendored
Normal file
35
vendor/github.com/onsi/gomega/matchers/assignable_to_type_of_matcher.go
generated
vendored
Normal file
@ -0,0 +1,35 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type AssignableToTypeOfMatcher struct {
|
||||
Expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *AssignableToTypeOfMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if actual == nil && matcher.Expected == nil {
|
||||
return false, fmt.Errorf("Refusing to compare <nil> to <nil>.\nBe explicit and use BeNil() instead. This is to avoid mistakes where both sides of an assertion are erroneously uninitialized.")
|
||||
} else if matcher.Expected == nil {
|
||||
return false, fmt.Errorf("Refusing to compare type to <nil>.\nBe explicit and use BeNil() instead. This is to avoid mistakes where both sides of an assertion are erroneously uninitialized.")
|
||||
} else if actual == nil {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
actualType := reflect.TypeOf(actual)
|
||||
expectedType := reflect.TypeOf(matcher.Expected)
|
||||
|
||||
return actualType.AssignableTo(expectedType), nil
|
||||
}
|
||||
|
||||
func (matcher *AssignableToTypeOfMatcher) FailureMessage(actual interface{}) string {
|
||||
return format.Message(actual, fmt.Sprintf("to be assignable to the type: %T", matcher.Expected))
|
||||
}
|
||||
|
||||
func (matcher *AssignableToTypeOfMatcher) NegatedFailureMessage(actual interface{}) string {
|
||||
return format.Message(actual, fmt.Sprintf("not to be assignable to the type: %T", matcher.Expected))
|
||||
}
|
14
vendor/github.com/onsi/gomega/matchers/attributes_slice.go
generated
vendored
Normal file
14
vendor/github.com/onsi/gomega/matchers/attributes_slice.go
generated
vendored
Normal file
@ -0,0 +1,14 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"strings"
|
||||
)
|
||||
|
||||
type attributesSlice []xml.Attr
|
||||
|
||||
func (attrs attributesSlice) Len() int { return len(attrs) }
|
||||
func (attrs attributesSlice) Less(i, j int) bool {
|
||||
return strings.Compare(attrs[i].Name.Local, attrs[j].Name.Local) == -1
|
||||
}
|
||||
func (attrs attributesSlice) Swap(i, j int) { attrs[i], attrs[j] = attrs[j], attrs[i] }
|
54
vendor/github.com/onsi/gomega/matchers/be_a_directory.go
generated
vendored
Normal file
54
vendor/github.com/onsi/gomega/matchers/be_a_directory.go
generated
vendored
Normal file
@ -0,0 +1,54 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type notADirectoryError struct {
|
||||
os.FileInfo
|
||||
}
|
||||
|
||||
func (t notADirectoryError) Error() string {
|
||||
fileInfo := os.FileInfo(t)
|
||||
switch {
|
||||
case fileInfo.Mode().IsRegular():
|
||||
return "file is a regular file"
|
||||
default:
|
||||
return fmt.Sprintf("file mode is: %s", fileInfo.Mode().String())
|
||||
}
|
||||
}
|
||||
|
||||
type BeADirectoryMatcher struct {
|
||||
expected interface{}
|
||||
err error
|
||||
}
|
||||
|
||||
func (matcher *BeADirectoryMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualFilename, ok := actual.(string)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("BeADirectoryMatcher matcher expects a file path")
|
||||
}
|
||||
|
||||
fileInfo, err := os.Stat(actualFilename)
|
||||
if err != nil {
|
||||
matcher.err = err
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if !fileInfo.Mode().IsDir() {
|
||||
matcher.err = notADirectoryError{fileInfo}
|
||||
return false, nil
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (matcher *BeADirectoryMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("to be a directory: %s", matcher.err))
|
||||
}
|
||||
|
||||
func (matcher *BeADirectoryMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("not be a directory"))
|
||||
}
|
54
vendor/github.com/onsi/gomega/matchers/be_a_regular_file.go
generated
vendored
Normal file
54
vendor/github.com/onsi/gomega/matchers/be_a_regular_file.go
generated
vendored
Normal file
@ -0,0 +1,54 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type notARegularFileError struct {
|
||||
os.FileInfo
|
||||
}
|
||||
|
||||
func (t notARegularFileError) Error() string {
|
||||
fileInfo := os.FileInfo(t)
|
||||
switch {
|
||||
case fileInfo.IsDir():
|
||||
return "file is a directory"
|
||||
default:
|
||||
return fmt.Sprintf("file mode is: %s", fileInfo.Mode().String())
|
||||
}
|
||||
}
|
||||
|
||||
type BeARegularFileMatcher struct {
|
||||
expected interface{}
|
||||
err error
|
||||
}
|
||||
|
||||
func (matcher *BeARegularFileMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualFilename, ok := actual.(string)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("BeARegularFileMatcher matcher expects a file path")
|
||||
}
|
||||
|
||||
fileInfo, err := os.Stat(actualFilename)
|
||||
if err != nil {
|
||||
matcher.err = err
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if !fileInfo.Mode().IsRegular() {
|
||||
matcher.err = notARegularFileError{fileInfo}
|
||||
return false, nil
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (matcher *BeARegularFileMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("to be a regular file: %s", matcher.err))
|
||||
}
|
||||
|
||||
func (matcher *BeARegularFileMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("not be a regular file"))
|
||||
}
|
38
vendor/github.com/onsi/gomega/matchers/be_an_existing_file.go
generated
vendored
Normal file
38
vendor/github.com/onsi/gomega/matchers/be_an_existing_file.go
generated
vendored
Normal file
@ -0,0 +1,38 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeAnExistingFileMatcher struct {
|
||||
expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *BeAnExistingFileMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualFilename, ok := actual.(string)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("BeAnExistingFileMatcher matcher expects a file path")
|
||||
}
|
||||
|
||||
if _, err = os.Stat(actualFilename); err != nil {
|
||||
switch {
|
||||
case os.IsNotExist(err):
|
||||
return false, nil
|
||||
default:
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (matcher *BeAnExistingFileMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("to exist"))
|
||||
}
|
||||
|
||||
func (matcher *BeAnExistingFileMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("not to exist"))
|
||||
}
|
46
vendor/github.com/onsi/gomega/matchers/be_closed_matcher.go
generated
vendored
Normal file
46
vendor/github.com/onsi/gomega/matchers/be_closed_matcher.go
generated
vendored
Normal file
@ -0,0 +1,46 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeClosedMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeClosedMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isChan(actual) {
|
||||
return false, fmt.Errorf("BeClosed matcher expects a channel. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
channelType := reflect.TypeOf(actual)
|
||||
channelValue := reflect.ValueOf(actual)
|
||||
|
||||
if channelType.ChanDir() == reflect.SendDir {
|
||||
return false, fmt.Errorf("BeClosed matcher cannot determine if a send-only channel is closed or open. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
winnerIndex, _, open := reflect.Select([]reflect.SelectCase{
|
||||
{Dir: reflect.SelectRecv, Chan: channelValue},
|
||||
{Dir: reflect.SelectDefault},
|
||||
})
|
||||
|
||||
var closed bool
|
||||
if winnerIndex == 0 {
|
||||
closed = !open
|
||||
} else if winnerIndex == 1 {
|
||||
closed = false
|
||||
}
|
||||
|
||||
return closed, nil
|
||||
}
|
||||
|
||||
func (matcher *BeClosedMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be closed")
|
||||
}
|
||||
|
||||
func (matcher *BeClosedMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be open")
|
||||
}
|
27
vendor/github.com/onsi/gomega/matchers/be_empty_matcher.go
generated
vendored
Normal file
27
vendor/github.com/onsi/gomega/matchers/be_empty_matcher.go
generated
vendored
Normal file
@ -0,0 +1,27 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeEmptyMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeEmptyMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
length, ok := lengthOf(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("BeEmpty matcher expects a string/array/map/channel/slice. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return length == 0, nil
|
||||
}
|
||||
|
||||
func (matcher *BeEmptyMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be empty")
|
||||
}
|
||||
|
||||
func (matcher *BeEmptyMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be empty")
|
||||
}
|
34
vendor/github.com/onsi/gomega/matchers/be_equivalent_to_matcher.go
generated
vendored
Normal file
34
vendor/github.com/onsi/gomega/matchers/be_equivalent_to_matcher.go
generated
vendored
Normal file
@ -0,0 +1,34 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeEquivalentToMatcher struct {
|
||||
Expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *BeEquivalentToMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if actual == nil && matcher.Expected == nil {
|
||||
return false, fmt.Errorf("Both actual and expected must not be nil.")
|
||||
}
|
||||
|
||||
convertedActual := actual
|
||||
|
||||
if actual != nil && matcher.Expected != nil && reflect.TypeOf(actual).ConvertibleTo(reflect.TypeOf(matcher.Expected)) {
|
||||
convertedActual = reflect.ValueOf(actual).Convert(reflect.TypeOf(matcher.Expected)).Interface()
|
||||
}
|
||||
|
||||
return reflect.DeepEqual(convertedActual, matcher.Expected), nil
|
||||
}
|
||||
|
||||
func (matcher *BeEquivalentToMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be equivalent to", matcher.Expected)
|
||||
}
|
||||
|
||||
func (matcher *BeEquivalentToMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be equivalent to", matcher.Expected)
|
||||
}
|
26
vendor/github.com/onsi/gomega/matchers/be_false_matcher.go
generated
vendored
Normal file
26
vendor/github.com/onsi/gomega/matchers/be_false_matcher.go
generated
vendored
Normal file
@ -0,0 +1,26 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeFalseMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeFalseMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isBool(actual) {
|
||||
return false, fmt.Errorf("Expected a boolean. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return actual == false, nil
|
||||
}
|
||||
|
||||
func (matcher *BeFalseMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be false")
|
||||
}
|
||||
|
||||
func (matcher *BeFalseMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be false")
|
||||
}
|
37
vendor/github.com/onsi/gomega/matchers/be_identical_to.go
generated
vendored
Normal file
37
vendor/github.com/onsi/gomega/matchers/be_identical_to.go
generated
vendored
Normal file
@ -0,0 +1,37 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"runtime"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeIdenticalToMatcher struct {
|
||||
Expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *BeIdenticalToMatcher) Match(actual interface{}) (success bool, matchErr error) {
|
||||
if actual == nil && matcher.Expected == nil {
|
||||
return false, fmt.Errorf("Refusing to compare <nil> to <nil>.\nBe explicit and use BeNil() instead. This is to avoid mistakes where both sides of an assertion are erroneously uninitialized.")
|
||||
}
|
||||
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
if _, ok := r.(runtime.Error); ok {
|
||||
success = false
|
||||
matchErr = nil
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return actual == matcher.Expected, nil
|
||||
}
|
||||
|
||||
func (matcher *BeIdenticalToMatcher) FailureMessage(actual interface{}) string {
|
||||
return format.Message(actual, "to be identical to", matcher.Expected)
|
||||
}
|
||||
|
||||
func (matcher *BeIdenticalToMatcher) NegatedFailureMessage(actual interface{}) string {
|
||||
return format.Message(actual, "not to be identical to", matcher.Expected)
|
||||
}
|
18
vendor/github.com/onsi/gomega/matchers/be_nil_matcher.go
generated
vendored
Normal file
18
vendor/github.com/onsi/gomega/matchers/be_nil_matcher.go
generated
vendored
Normal file
@ -0,0 +1,18 @@
|
||||
package matchers
|
||||
|
||||
import "github.com/onsi/gomega/format"
|
||||
|
||||
type BeNilMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeNilMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
return isNil(actual), nil
|
||||
}
|
||||
|
||||
func (matcher *BeNilMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be nil")
|
||||
}
|
||||
|
||||
func (matcher *BeNilMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be nil")
|
||||
}
|
132
vendor/github.com/onsi/gomega/matchers/be_numerically_matcher.go
generated
vendored
Normal file
132
vendor/github.com/onsi/gomega/matchers/be_numerically_matcher.go
generated
vendored
Normal file
@ -0,0 +1,132 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeNumericallyMatcher struct {
|
||||
Comparator string
|
||||
CompareTo []interface{}
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return matcher.FormatFailureMessage(actual, false)
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return matcher.FormatFailureMessage(actual, true)
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) FormatFailureMessage(actual interface{}, negated bool) (message string) {
|
||||
if len(matcher.CompareTo) == 1 {
|
||||
message = fmt.Sprintf("to be %s", matcher.Comparator)
|
||||
} else {
|
||||
message = fmt.Sprintf("to be within %v of %s", matcher.CompareTo[1], matcher.Comparator)
|
||||
}
|
||||
if negated {
|
||||
message = "not " + message
|
||||
}
|
||||
return format.Message(actual, message, matcher.CompareTo[0])
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if len(matcher.CompareTo) == 0 || len(matcher.CompareTo) > 2 {
|
||||
return false, fmt.Errorf("BeNumerically requires 1 or 2 CompareTo arguments. Got:\n%s", format.Object(matcher.CompareTo, 1))
|
||||
}
|
||||
if !isNumber(actual) {
|
||||
return false, fmt.Errorf("Expected a number. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
if !isNumber(matcher.CompareTo[0]) {
|
||||
return false, fmt.Errorf("Expected a number. Got:\n%s", format.Object(matcher.CompareTo[0], 1))
|
||||
}
|
||||
if len(matcher.CompareTo) == 2 && !isNumber(matcher.CompareTo[1]) {
|
||||
return false, fmt.Errorf("Expected a number. Got:\n%s", format.Object(matcher.CompareTo[0], 1))
|
||||
}
|
||||
|
||||
switch matcher.Comparator {
|
||||
case "==", "~", ">", ">=", "<", "<=":
|
||||
default:
|
||||
return false, fmt.Errorf("Unknown comparator: %s", matcher.Comparator)
|
||||
}
|
||||
|
||||
if isFloat(actual) || isFloat(matcher.CompareTo[0]) {
|
||||
var secondOperand float64 = 1e-8
|
||||
if len(matcher.CompareTo) == 2 {
|
||||
secondOperand = toFloat(matcher.CompareTo[1])
|
||||
}
|
||||
success = matcher.matchFloats(toFloat(actual), toFloat(matcher.CompareTo[0]), secondOperand)
|
||||
} else if isInteger(actual) {
|
||||
var secondOperand int64 = 0
|
||||
if len(matcher.CompareTo) == 2 {
|
||||
secondOperand = toInteger(matcher.CompareTo[1])
|
||||
}
|
||||
success = matcher.matchIntegers(toInteger(actual), toInteger(matcher.CompareTo[0]), secondOperand)
|
||||
} else if isUnsignedInteger(actual) {
|
||||
var secondOperand uint64 = 0
|
||||
if len(matcher.CompareTo) == 2 {
|
||||
secondOperand = toUnsignedInteger(matcher.CompareTo[1])
|
||||
}
|
||||
success = matcher.matchUnsignedIntegers(toUnsignedInteger(actual), toUnsignedInteger(matcher.CompareTo[0]), secondOperand)
|
||||
} else {
|
||||
return false, fmt.Errorf("Failed to compare:\n%s\n%s:\n%s", format.Object(actual, 1), matcher.Comparator, format.Object(matcher.CompareTo[0], 1))
|
||||
}
|
||||
|
||||
return success, nil
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) matchIntegers(actual, compareTo, threshold int64) (success bool) {
|
||||
switch matcher.Comparator {
|
||||
case "==", "~":
|
||||
diff := actual - compareTo
|
||||
return -threshold <= diff && diff <= threshold
|
||||
case ">":
|
||||
return (actual > compareTo)
|
||||
case ">=":
|
||||
return (actual >= compareTo)
|
||||
case "<":
|
||||
return (actual < compareTo)
|
||||
case "<=":
|
||||
return (actual <= compareTo)
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) matchUnsignedIntegers(actual, compareTo, threshold uint64) (success bool) {
|
||||
switch matcher.Comparator {
|
||||
case "==", "~":
|
||||
if actual < compareTo {
|
||||
actual, compareTo = compareTo, actual
|
||||
}
|
||||
return actual-compareTo <= threshold
|
||||
case ">":
|
||||
return (actual > compareTo)
|
||||
case ">=":
|
||||
return (actual >= compareTo)
|
||||
case "<":
|
||||
return (actual < compareTo)
|
||||
case "<=":
|
||||
return (actual <= compareTo)
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (matcher *BeNumericallyMatcher) matchFloats(actual, compareTo, threshold float64) (success bool) {
|
||||
switch matcher.Comparator {
|
||||
case "~":
|
||||
return math.Abs(actual-compareTo) <= threshold
|
||||
case "==":
|
||||
return (actual == compareTo)
|
||||
case ">":
|
||||
return (actual > compareTo)
|
||||
case ">=":
|
||||
return (actual >= compareTo)
|
||||
case "<":
|
||||
return (actual < compareTo)
|
||||
case "<=":
|
||||
return (actual <= compareTo)
|
||||
}
|
||||
return false
|
||||
}
|
71
vendor/github.com/onsi/gomega/matchers/be_sent_matcher.go
generated
vendored
Normal file
71
vendor/github.com/onsi/gomega/matchers/be_sent_matcher.go
generated
vendored
Normal file
@ -0,0 +1,71 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeSentMatcher struct {
|
||||
Arg interface{}
|
||||
channelClosed bool
|
||||
}
|
||||
|
||||
func (matcher *BeSentMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isChan(actual) {
|
||||
return false, fmt.Errorf("BeSent expects a channel. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
channelType := reflect.TypeOf(actual)
|
||||
channelValue := reflect.ValueOf(actual)
|
||||
|
||||
if channelType.ChanDir() == reflect.RecvDir {
|
||||
return false, fmt.Errorf("BeSent matcher cannot be passed a receive-only channel. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
argType := reflect.TypeOf(matcher.Arg)
|
||||
assignable := argType.AssignableTo(channelType.Elem())
|
||||
|
||||
if !assignable {
|
||||
return false, fmt.Errorf("Cannot pass:\n%s to the channel:\n%s\nThe types don't match.", format.Object(matcher.Arg, 1), format.Object(actual, 1))
|
||||
}
|
||||
|
||||
argValue := reflect.ValueOf(matcher.Arg)
|
||||
|
||||
defer func() {
|
||||
if e := recover(); e != nil {
|
||||
success = false
|
||||
err = fmt.Errorf("Cannot send to a closed channel")
|
||||
matcher.channelClosed = true
|
||||
}
|
||||
}()
|
||||
|
||||
winnerIndex, _, _ := reflect.Select([]reflect.SelectCase{
|
||||
{Dir: reflect.SelectSend, Chan: channelValue, Send: argValue},
|
||||
{Dir: reflect.SelectDefault},
|
||||
})
|
||||
|
||||
var didSend bool
|
||||
if winnerIndex == 0 {
|
||||
didSend = true
|
||||
}
|
||||
|
||||
return didSend, nil
|
||||
}
|
||||
|
||||
func (matcher *BeSentMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to send:", matcher.Arg)
|
||||
}
|
||||
|
||||
func (matcher *BeSentMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to send:", matcher.Arg)
|
||||
}
|
||||
|
||||
func (matcher *BeSentMatcher) MatchMayChangeInTheFuture(actual interface{}) bool {
|
||||
if !isChan(actual) {
|
||||
return false
|
||||
}
|
||||
|
||||
return !matcher.channelClosed
|
||||
}
|
66
vendor/github.com/onsi/gomega/matchers/be_temporally_matcher.go
generated
vendored
Normal file
66
vendor/github.com/onsi/gomega/matchers/be_temporally_matcher.go
generated
vendored
Normal file
@ -0,0 +1,66 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeTemporallyMatcher struct {
|
||||
Comparator string
|
||||
CompareTo time.Time
|
||||
Threshold []time.Duration
|
||||
}
|
||||
|
||||
func (matcher *BeTemporallyMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("to be %s", matcher.Comparator), matcher.CompareTo)
|
||||
}
|
||||
|
||||
func (matcher *BeTemporallyMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("not to be %s", matcher.Comparator), matcher.CompareTo)
|
||||
}
|
||||
|
||||
func (matcher *BeTemporallyMatcher) Match(actual interface{}) (bool, error) {
|
||||
// predicate to test for time.Time type
|
||||
isTime := func(t interface{}) bool {
|
||||
_, ok := t.(time.Time)
|
||||
return ok
|
||||
}
|
||||
|
||||
if !isTime(actual) {
|
||||
return false, fmt.Errorf("Expected a time.Time. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
switch matcher.Comparator {
|
||||
case "==", "~", ">", ">=", "<", "<=":
|
||||
default:
|
||||
return false, fmt.Errorf("Unknown comparator: %s", matcher.Comparator)
|
||||
}
|
||||
|
||||
var threshold = time.Millisecond
|
||||
if len(matcher.Threshold) == 1 {
|
||||
threshold = matcher.Threshold[0]
|
||||
}
|
||||
|
||||
return matcher.matchTimes(actual.(time.Time), matcher.CompareTo, threshold), nil
|
||||
}
|
||||
|
||||
func (matcher *BeTemporallyMatcher) matchTimes(actual, compareTo time.Time, threshold time.Duration) (success bool) {
|
||||
switch matcher.Comparator {
|
||||
case "==":
|
||||
return actual.Equal(compareTo)
|
||||
case "~":
|
||||
diff := actual.Sub(compareTo)
|
||||
return -threshold <= diff && diff <= threshold
|
||||
case ">":
|
||||
return actual.After(compareTo)
|
||||
case ">=":
|
||||
return !actual.Before(compareTo)
|
||||
case "<":
|
||||
return actual.Before(compareTo)
|
||||
case "<=":
|
||||
return !actual.After(compareTo)
|
||||
}
|
||||
return false
|
||||
}
|
26
vendor/github.com/onsi/gomega/matchers/be_true_matcher.go
generated
vendored
Normal file
26
vendor/github.com/onsi/gomega/matchers/be_true_matcher.go
generated
vendored
Normal file
@ -0,0 +1,26 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeTrueMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeTrueMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isBool(actual) {
|
||||
return false, fmt.Errorf("Expected a boolean. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return actual.(bool), nil
|
||||
}
|
||||
|
||||
func (matcher *BeTrueMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be true")
|
||||
}
|
||||
|
||||
func (matcher *BeTrueMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be true")
|
||||
}
|
28
vendor/github.com/onsi/gomega/matchers/be_zero_matcher.go
generated
vendored
Normal file
28
vendor/github.com/onsi/gomega/matchers/be_zero_matcher.go
generated
vendored
Normal file
@ -0,0 +1,28 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type BeZeroMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *BeZeroMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if actual == nil {
|
||||
return true, nil
|
||||
}
|
||||
zeroValue := reflect.Zero(reflect.TypeOf(actual)).Interface()
|
||||
|
||||
return reflect.DeepEqual(zeroValue, actual), nil
|
||||
|
||||
}
|
||||
|
||||
func (matcher *BeZeroMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to be zero-valued")
|
||||
}
|
||||
|
||||
func (matcher *BeZeroMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to be zero-valued")
|
||||
}
|
80
vendor/github.com/onsi/gomega/matchers/consist_of.go
generated
vendored
Normal file
80
vendor/github.com/onsi/gomega/matchers/consist_of.go
generated
vendored
Normal file
@ -0,0 +1,80 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
"github.com/onsi/gomega/matchers/support/goraph/bipartitegraph"
|
||||
)
|
||||
|
||||
type ConsistOfMatcher struct {
|
||||
Elements []interface{}
|
||||
}
|
||||
|
||||
func (matcher *ConsistOfMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isArrayOrSlice(actual) && !isMap(actual) {
|
||||
return false, fmt.Errorf("ConsistOf matcher expects an array/slice/map. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
elements := matcher.Elements
|
||||
if len(matcher.Elements) == 1 && isArrayOrSlice(matcher.Elements[0]) {
|
||||
elements = []interface{}{}
|
||||
value := reflect.ValueOf(matcher.Elements[0])
|
||||
for i := 0; i < value.Len(); i++ {
|
||||
elements = append(elements, value.Index(i).Interface())
|
||||
}
|
||||
}
|
||||
|
||||
matchers := []interface{}{}
|
||||
for _, element := range elements {
|
||||
matcher, isMatcher := element.(omegaMatcher)
|
||||
if !isMatcher {
|
||||
matcher = &EqualMatcher{Expected: element}
|
||||
}
|
||||
matchers = append(matchers, matcher)
|
||||
}
|
||||
|
||||
values := matcher.valuesOf(actual)
|
||||
|
||||
if len(values) != len(matchers) {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
neighbours := func(v, m interface{}) (bool, error) {
|
||||
match, err := m.(omegaMatcher).Match(v)
|
||||
return match && err == nil, nil
|
||||
}
|
||||
|
||||
bipartiteGraph, err := bipartitegraph.NewBipartiteGraph(values, matchers, neighbours)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return len(bipartiteGraph.LargestMatching()) == len(values), nil
|
||||
}
|
||||
|
||||
func (matcher *ConsistOfMatcher) valuesOf(actual interface{}) []interface{} {
|
||||
value := reflect.ValueOf(actual)
|
||||
values := []interface{}{}
|
||||
if isMap(actual) {
|
||||
keys := value.MapKeys()
|
||||
for i := 0; i < value.Len(); i++ {
|
||||
values = append(values, value.MapIndex(keys[i]).Interface())
|
||||
}
|
||||
} else {
|
||||
for i := 0; i < value.Len(); i++ {
|
||||
values = append(values, value.Index(i).Interface())
|
||||
}
|
||||
}
|
||||
|
||||
return values
|
||||
}
|
||||
|
||||
func (matcher *ConsistOfMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to consist of", matcher.Elements)
|
||||
}
|
||||
|
||||
func (matcher *ConsistOfMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to consist of", matcher.Elements)
|
||||
}
|
56
vendor/github.com/onsi/gomega/matchers/contain_element_matcher.go
generated
vendored
Normal file
56
vendor/github.com/onsi/gomega/matchers/contain_element_matcher.go
generated
vendored
Normal file
@ -0,0 +1,56 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type ContainElementMatcher struct {
|
||||
Element interface{}
|
||||
}
|
||||
|
||||
func (matcher *ContainElementMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isArrayOrSlice(actual) && !isMap(actual) {
|
||||
return false, fmt.Errorf("ContainElement matcher expects an array/slice/map. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
elemMatcher, elementIsMatcher := matcher.Element.(omegaMatcher)
|
||||
if !elementIsMatcher {
|
||||
elemMatcher = &EqualMatcher{Expected: matcher.Element}
|
||||
}
|
||||
|
||||
value := reflect.ValueOf(actual)
|
||||
var keys []reflect.Value
|
||||
if isMap(actual) {
|
||||
keys = value.MapKeys()
|
||||
}
|
||||
var lastError error
|
||||
for i := 0; i < value.Len(); i++ {
|
||||
var success bool
|
||||
var err error
|
||||
if isMap(actual) {
|
||||
success, err = elemMatcher.Match(value.MapIndex(keys[i]).Interface())
|
||||
} else {
|
||||
success, err = elemMatcher.Match(value.Index(i).Interface())
|
||||
}
|
||||
if err != nil {
|
||||
lastError = err
|
||||
continue
|
||||
}
|
||||
if success {
|
||||
return true, nil
|
||||
}
|
||||
}
|
||||
|
||||
return false, lastError
|
||||
}
|
||||
|
||||
func (matcher *ContainElementMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to contain element matching", matcher.Element)
|
||||
}
|
||||
|
||||
func (matcher *ContainElementMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to contain element matching", matcher.Element)
|
||||
}
|
38
vendor/github.com/onsi/gomega/matchers/contain_substring_matcher.go
generated
vendored
Normal file
38
vendor/github.com/onsi/gomega/matchers/contain_substring_matcher.go
generated
vendored
Normal file
@ -0,0 +1,38 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type ContainSubstringMatcher struct {
|
||||
Substr string
|
||||
Args []interface{}
|
||||
}
|
||||
|
||||
func (matcher *ContainSubstringMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("ContainSubstring matcher requires a string or stringer. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return strings.Contains(actualString, matcher.stringToMatch()), nil
|
||||
}
|
||||
|
||||
func (matcher *ContainSubstringMatcher) stringToMatch() string {
|
||||
stringToMatch := matcher.Substr
|
||||
if len(matcher.Args) > 0 {
|
||||
stringToMatch = fmt.Sprintf(matcher.Substr, matcher.Args...)
|
||||
}
|
||||
return stringToMatch
|
||||
}
|
||||
|
||||
func (matcher *ContainSubstringMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to contain substring", matcher.stringToMatch())
|
||||
}
|
||||
|
||||
func (matcher *ContainSubstringMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to contain substring", matcher.stringToMatch())
|
||||
}
|
42
vendor/github.com/onsi/gomega/matchers/equal_matcher.go
generated
vendored
Normal file
42
vendor/github.com/onsi/gomega/matchers/equal_matcher.go
generated
vendored
Normal file
@ -0,0 +1,42 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type EqualMatcher struct {
|
||||
Expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *EqualMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if actual == nil && matcher.Expected == nil {
|
||||
return false, fmt.Errorf("Refusing to compare <nil> to <nil>.\nBe explicit and use BeNil() instead. This is to avoid mistakes where both sides of an assertion are erroneously uninitialized.")
|
||||
}
|
||||
// Shortcut for byte slices.
|
||||
// Comparing long byte slices with reflect.DeepEqual is very slow,
|
||||
// so use bytes.Equal if actual and expected are both byte slices.
|
||||
if actualByteSlice, ok := actual.([]byte); ok {
|
||||
if expectedByteSlice, ok := matcher.Expected.([]byte); ok {
|
||||
return bytes.Equal(actualByteSlice, expectedByteSlice), nil
|
||||
}
|
||||
}
|
||||
return reflect.DeepEqual(actual, matcher.Expected), nil
|
||||
}
|
||||
|
||||
func (matcher *EqualMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
actualString, actualOK := actual.(string)
|
||||
expectedString, expectedOK := matcher.Expected.(string)
|
||||
if actualOK && expectedOK {
|
||||
return format.MessageWithDiff(actualString, "to equal", expectedString)
|
||||
}
|
||||
|
||||
return format.Message(actual, "to equal", matcher.Expected)
|
||||
}
|
||||
|
||||
func (matcher *EqualMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to equal", matcher.Expected)
|
||||
}
|
28
vendor/github.com/onsi/gomega/matchers/have_cap_matcher.go
generated
vendored
Normal file
28
vendor/github.com/onsi/gomega/matchers/have_cap_matcher.go
generated
vendored
Normal file
@ -0,0 +1,28 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveCapMatcher struct {
|
||||
Count int
|
||||
}
|
||||
|
||||
func (matcher *HaveCapMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
length, ok := capOf(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("HaveCap matcher expects a array/channel/slice. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return length == matcher.Count, nil
|
||||
}
|
||||
|
||||
func (matcher *HaveCapMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected\n%s\nto have capacity %d", format.Object(actual, 1), matcher.Count)
|
||||
}
|
||||
|
||||
func (matcher *HaveCapMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected\n%s\nnot to have capacity %d", format.Object(actual, 1), matcher.Count)
|
||||
}
|
54
vendor/github.com/onsi/gomega/matchers/have_key_matcher.go
generated
vendored
Normal file
54
vendor/github.com/onsi/gomega/matchers/have_key_matcher.go
generated
vendored
Normal file
@ -0,0 +1,54 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveKeyMatcher struct {
|
||||
Key interface{}
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isMap(actual) {
|
||||
return false, fmt.Errorf("HaveKey matcher expects a map. Got:%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
keyMatcher, keyIsMatcher := matcher.Key.(omegaMatcher)
|
||||
if !keyIsMatcher {
|
||||
keyMatcher = &EqualMatcher{Expected: matcher.Key}
|
||||
}
|
||||
|
||||
keys := reflect.ValueOf(actual).MapKeys()
|
||||
for i := 0; i < len(keys); i++ {
|
||||
success, err := keyMatcher.Match(keys[i].Interface())
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("HaveKey's key matcher failed with:\n%s%s", format.Indent, err.Error())
|
||||
}
|
||||
if success {
|
||||
return true, nil
|
||||
}
|
||||
}
|
||||
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
switch matcher.Key.(type) {
|
||||
case omegaMatcher:
|
||||
return format.Message(actual, "to have key matching", matcher.Key)
|
||||
default:
|
||||
return format.Message(actual, "to have key", matcher.Key)
|
||||
}
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
switch matcher.Key.(type) {
|
||||
case omegaMatcher:
|
||||
return format.Message(actual, "not to have key matching", matcher.Key)
|
||||
default:
|
||||
return format.Message(actual, "not to have key", matcher.Key)
|
||||
}
|
||||
}
|
74
vendor/github.com/onsi/gomega/matchers/have_key_with_value_matcher.go
generated
vendored
Normal file
74
vendor/github.com/onsi/gomega/matchers/have_key_with_value_matcher.go
generated
vendored
Normal file
@ -0,0 +1,74 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveKeyWithValueMatcher struct {
|
||||
Key interface{}
|
||||
Value interface{}
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyWithValueMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isMap(actual) {
|
||||
return false, fmt.Errorf("HaveKeyWithValue matcher expects a map. Got:%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
keyMatcher, keyIsMatcher := matcher.Key.(omegaMatcher)
|
||||
if !keyIsMatcher {
|
||||
keyMatcher = &EqualMatcher{Expected: matcher.Key}
|
||||
}
|
||||
|
||||
valueMatcher, valueIsMatcher := matcher.Value.(omegaMatcher)
|
||||
if !valueIsMatcher {
|
||||
valueMatcher = &EqualMatcher{Expected: matcher.Value}
|
||||
}
|
||||
|
||||
keys := reflect.ValueOf(actual).MapKeys()
|
||||
for i := 0; i < len(keys); i++ {
|
||||
success, err := keyMatcher.Match(keys[i].Interface())
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("HaveKeyWithValue's key matcher failed with:\n%s%s", format.Indent, err.Error())
|
||||
}
|
||||
if success {
|
||||
actualValue := reflect.ValueOf(actual).MapIndex(keys[i])
|
||||
success, err := valueMatcher.Match(actualValue.Interface())
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("HaveKeyWithValue's value matcher failed with:\n%s%s", format.Indent, err.Error())
|
||||
}
|
||||
return success, nil
|
||||
}
|
||||
}
|
||||
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyWithValueMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
str := "to have {key: value}"
|
||||
if _, ok := matcher.Key.(omegaMatcher); ok {
|
||||
str += " matching"
|
||||
} else if _, ok := matcher.Value.(omegaMatcher); ok {
|
||||
str += " matching"
|
||||
}
|
||||
|
||||
expect := make(map[interface{}]interface{}, 1)
|
||||
expect[matcher.Key] = matcher.Value
|
||||
return format.Message(actual, str, expect)
|
||||
}
|
||||
|
||||
func (matcher *HaveKeyWithValueMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
kStr := "not to have key"
|
||||
if _, ok := matcher.Key.(omegaMatcher); ok {
|
||||
kStr = "not to have key matching"
|
||||
}
|
||||
|
||||
vStr := "or that key's value not be"
|
||||
if _, ok := matcher.Value.(omegaMatcher); ok {
|
||||
vStr = "or to have that key's value not matching"
|
||||
}
|
||||
|
||||
return format.Message(actual, kStr, matcher.Key, vStr, matcher.Value)
|
||||
}
|
28
vendor/github.com/onsi/gomega/matchers/have_len_matcher.go
generated
vendored
Normal file
28
vendor/github.com/onsi/gomega/matchers/have_len_matcher.go
generated
vendored
Normal file
@ -0,0 +1,28 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveLenMatcher struct {
|
||||
Count int
|
||||
}
|
||||
|
||||
func (matcher *HaveLenMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
length, ok := lengthOf(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("HaveLen matcher expects a string/array/map/channel/slice. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
return length == matcher.Count, nil
|
||||
}
|
||||
|
||||
func (matcher *HaveLenMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected\n%s\nto have length %d", format.Object(actual, 1), matcher.Count)
|
||||
}
|
||||
|
||||
func (matcher *HaveLenMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected\n%s\nnot to have length %d", format.Object(actual, 1), matcher.Count)
|
||||
}
|
33
vendor/github.com/onsi/gomega/matchers/have_occurred_matcher.go
generated
vendored
Normal file
33
vendor/github.com/onsi/gomega/matchers/have_occurred_matcher.go
generated
vendored
Normal file
@ -0,0 +1,33 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveOccurredMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *HaveOccurredMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
// is purely nil?
|
||||
if actual == nil {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
// must be an 'error' type
|
||||
if !isError(actual) {
|
||||
return false, fmt.Errorf("Expected an error-type. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
// must be non-nil (or a pointer to a non-nil)
|
||||
return !isNil(actual), nil
|
||||
}
|
||||
|
||||
func (matcher *HaveOccurredMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected an error to have occurred. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
func (matcher *HaveOccurredMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Unexpected error:\n%s\n%s\n%s", format.Object(actual, 1), format.IndentString(actual.(error).Error(), 1), "occurred")
|
||||
}
|
36
vendor/github.com/onsi/gomega/matchers/have_prefix_matcher.go
generated
vendored
Normal file
36
vendor/github.com/onsi/gomega/matchers/have_prefix_matcher.go
generated
vendored
Normal file
@ -0,0 +1,36 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HavePrefixMatcher struct {
|
||||
Prefix string
|
||||
Args []interface{}
|
||||
}
|
||||
|
||||
func (matcher *HavePrefixMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("HavePrefix matcher requires a string or stringer. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
prefix := matcher.prefix()
|
||||
return len(actualString) >= len(prefix) && actualString[0:len(prefix)] == prefix, nil
|
||||
}
|
||||
|
||||
func (matcher *HavePrefixMatcher) prefix() string {
|
||||
if len(matcher.Args) > 0 {
|
||||
return fmt.Sprintf(matcher.Prefix, matcher.Args...)
|
||||
}
|
||||
return matcher.Prefix
|
||||
}
|
||||
|
||||
func (matcher *HavePrefixMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to have prefix", matcher.prefix())
|
||||
}
|
||||
|
||||
func (matcher *HavePrefixMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to have prefix", matcher.prefix())
|
||||
}
|
36
vendor/github.com/onsi/gomega/matchers/have_suffix_matcher.go
generated
vendored
Normal file
36
vendor/github.com/onsi/gomega/matchers/have_suffix_matcher.go
generated
vendored
Normal file
@ -0,0 +1,36 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type HaveSuffixMatcher struct {
|
||||
Suffix string
|
||||
Args []interface{}
|
||||
}
|
||||
|
||||
func (matcher *HaveSuffixMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("HaveSuffix matcher requires a string or stringer. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
suffix := matcher.suffix()
|
||||
return len(actualString) >= len(suffix) && actualString[len(actualString)-len(suffix):] == suffix, nil
|
||||
}
|
||||
|
||||
func (matcher *HaveSuffixMatcher) suffix() string {
|
||||
if len(matcher.Args) > 0 {
|
||||
return fmt.Sprintf(matcher.Suffix, matcher.Args...)
|
||||
}
|
||||
return matcher.Suffix
|
||||
}
|
||||
|
||||
func (matcher *HaveSuffixMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to have suffix", matcher.suffix())
|
||||
}
|
||||
|
||||
func (matcher *HaveSuffixMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to have suffix", matcher.suffix())
|
||||
}
|
51
vendor/github.com/onsi/gomega/matchers/match_error_matcher.go
generated
vendored
Normal file
51
vendor/github.com/onsi/gomega/matchers/match_error_matcher.go
generated
vendored
Normal file
@ -0,0 +1,51 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type MatchErrorMatcher struct {
|
||||
Expected interface{}
|
||||
}
|
||||
|
||||
func (matcher *MatchErrorMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if isNil(actual) {
|
||||
return false, fmt.Errorf("Expected an error, got nil")
|
||||
}
|
||||
|
||||
if !isError(actual) {
|
||||
return false, fmt.Errorf("Expected an error. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
actualErr := actual.(error)
|
||||
|
||||
if isError(matcher.Expected) {
|
||||
return reflect.DeepEqual(actualErr, matcher.Expected), nil
|
||||
}
|
||||
|
||||
if isString(matcher.Expected) {
|
||||
return actualErr.Error() == matcher.Expected, nil
|
||||
}
|
||||
|
||||
var subMatcher omegaMatcher
|
||||
var hasSubMatcher bool
|
||||
if matcher.Expected != nil {
|
||||
subMatcher, hasSubMatcher = (matcher.Expected).(omegaMatcher)
|
||||
if hasSubMatcher {
|
||||
return subMatcher.Match(actualErr.Error())
|
||||
}
|
||||
}
|
||||
|
||||
return false, fmt.Errorf("MatchError must be passed an error, string, or Matcher that can match on strings. Got:\n%s", format.Object(matcher.Expected, 1))
|
||||
}
|
||||
|
||||
func (matcher *MatchErrorMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to match error", matcher.Expected)
|
||||
}
|
||||
|
||||
func (matcher *MatchErrorMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to match error", matcher.Expected)
|
||||
}
|
65
vendor/github.com/onsi/gomega/matchers/match_json_matcher.go
generated
vendored
Normal file
65
vendor/github.com/onsi/gomega/matchers/match_json_matcher.go
generated
vendored
Normal file
@ -0,0 +1,65 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type MatchJSONMatcher struct {
|
||||
JSONToMatch interface{}
|
||||
firstFailurePath []interface{}
|
||||
}
|
||||
|
||||
func (matcher *MatchJSONMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, expectedString, err := matcher.prettyPrint(actual)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
var aval interface{}
|
||||
var eval interface{}
|
||||
|
||||
// this is guarded by prettyPrint
|
||||
json.Unmarshal([]byte(actualString), &aval)
|
||||
json.Unmarshal([]byte(expectedString), &eval)
|
||||
var equal bool
|
||||
equal, matcher.firstFailurePath = deepEqual(aval, eval)
|
||||
return equal, nil
|
||||
}
|
||||
|
||||
func (matcher *MatchJSONMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.prettyPrint(actual)
|
||||
return formattedMessage(format.Message(actualString, "to match JSON of", expectedString), matcher.firstFailurePath)
|
||||
}
|
||||
|
||||
func (matcher *MatchJSONMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.prettyPrint(actual)
|
||||
return formattedMessage(format.Message(actualString, "not to match JSON of", expectedString), matcher.firstFailurePath)
|
||||
}
|
||||
|
||||
func (matcher *MatchJSONMatcher) prettyPrint(actual interface{}) (actualFormatted, expectedFormatted string, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchJSONMatcher matcher requires a string, stringer, or []byte. Got actual:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
expectedString, ok := toString(matcher.JSONToMatch)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchJSONMatcher matcher requires a string, stringer, or []byte. Got expected:\n%s", format.Object(matcher.JSONToMatch, 1))
|
||||
}
|
||||
|
||||
abuf := new(bytes.Buffer)
|
||||
ebuf := new(bytes.Buffer)
|
||||
|
||||
if err := json.Indent(abuf, []byte(actualString), "", " "); err != nil {
|
||||
return "", "", fmt.Errorf("Actual '%s' should be valid JSON, but it is not.\nUnderlying error:%s", actualString, err)
|
||||
}
|
||||
|
||||
if err := json.Indent(ebuf, []byte(expectedString), "", " "); err != nil {
|
||||
return "", "", fmt.Errorf("Expected '%s' should be valid JSON, but it is not.\nUnderlying error:%s", expectedString, err)
|
||||
}
|
||||
|
||||
return abuf.String(), ebuf.String(), nil
|
||||
}
|
43
vendor/github.com/onsi/gomega/matchers/match_regexp_matcher.go
generated
vendored
Normal file
43
vendor/github.com/onsi/gomega/matchers/match_regexp_matcher.go
generated
vendored
Normal file
@ -0,0 +1,43 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"regexp"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type MatchRegexpMatcher struct {
|
||||
Regexp string
|
||||
Args []interface{}
|
||||
}
|
||||
|
||||
func (matcher *MatchRegexpMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return false, fmt.Errorf("RegExp matcher requires a string or stringer.\nGot:%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
match, err := regexp.Match(matcher.regexp(), []byte(actualString))
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("RegExp match failed to compile with error:\n\t%s", err.Error())
|
||||
}
|
||||
|
||||
return match, nil
|
||||
}
|
||||
|
||||
func (matcher *MatchRegexpMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to match regular expression", matcher.regexp())
|
||||
}
|
||||
|
||||
func (matcher *MatchRegexpMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "not to match regular expression", matcher.regexp())
|
||||
}
|
||||
|
||||
func (matcher *MatchRegexpMatcher) regexp() string {
|
||||
re := matcher.Regexp
|
||||
if len(matcher.Args) > 0 {
|
||||
re = fmt.Sprintf(matcher.Regexp, matcher.Args...)
|
||||
}
|
||||
return re
|
||||
}
|
134
vendor/github.com/onsi/gomega/matchers/match_xml_matcher.go
generated
vendored
Normal file
134
vendor/github.com/onsi/gomega/matchers/match_xml_matcher.go
generated
vendored
Normal file
@ -0,0 +1,134 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"reflect"
|
||||
"sort"
|
||||
"strings"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
"golang.org/x/net/html/charset"
|
||||
)
|
||||
|
||||
type MatchXMLMatcher struct {
|
||||
XMLToMatch interface{}
|
||||
}
|
||||
|
||||
func (matcher *MatchXMLMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, expectedString, err := matcher.formattedPrint(actual)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
aval, err := parseXmlContent(actualString)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("Actual '%s' should be valid XML, but it is not.\nUnderlying error:%s", actualString, err)
|
||||
}
|
||||
|
||||
eval, err := parseXmlContent(expectedString)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("Expected '%s' should be valid XML, but it is not.\nUnderlying error:%s", expectedString, err)
|
||||
}
|
||||
|
||||
return reflect.DeepEqual(aval, eval), nil
|
||||
}
|
||||
|
||||
func (matcher *MatchXMLMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.formattedPrint(actual)
|
||||
return fmt.Sprintf("Expected\n%s\nto match XML of\n%s", actualString, expectedString)
|
||||
}
|
||||
|
||||
func (matcher *MatchXMLMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.formattedPrint(actual)
|
||||
return fmt.Sprintf("Expected\n%s\nnot to match XML of\n%s", actualString, expectedString)
|
||||
}
|
||||
|
||||
func (matcher *MatchXMLMatcher) formattedPrint(actual interface{}) (actualString, expectedString string, err error) {
|
||||
var ok bool
|
||||
actualString, ok = toString(actual)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchXMLMatcher matcher requires a string, stringer, or []byte. Got actual:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
expectedString, ok = toString(matcher.XMLToMatch)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchXMLMatcher matcher requires a string, stringer, or []byte. Got expected:\n%s", format.Object(matcher.XMLToMatch, 1))
|
||||
}
|
||||
return actualString, expectedString, nil
|
||||
}
|
||||
|
||||
func parseXmlContent(content string) (*xmlNode, error) {
|
||||
allNodes := []*xmlNode{}
|
||||
|
||||
dec := newXmlDecoder(strings.NewReader(content))
|
||||
for {
|
||||
tok, err := dec.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return nil, fmt.Errorf("failed to decode next token: %v", err)
|
||||
}
|
||||
|
||||
lastNodeIndex := len(allNodes) - 1
|
||||
var lastNode *xmlNode
|
||||
if len(allNodes) > 0 {
|
||||
lastNode = allNodes[lastNodeIndex]
|
||||
} else {
|
||||
lastNode = &xmlNode{}
|
||||
}
|
||||
|
||||
switch tok := tok.(type) {
|
||||
case xml.StartElement:
|
||||
attrs := attributesSlice(tok.Attr)
|
||||
sort.Sort(attrs)
|
||||
allNodes = append(allNodes, &xmlNode{XMLName: tok.Name, XMLAttr: tok.Attr})
|
||||
case xml.EndElement:
|
||||
if len(allNodes) > 1 {
|
||||
allNodes[lastNodeIndex-1].Nodes = append(allNodes[lastNodeIndex-1].Nodes, lastNode)
|
||||
allNodes = allNodes[:lastNodeIndex]
|
||||
}
|
||||
case xml.CharData:
|
||||
lastNode.Content = append(lastNode.Content, tok.Copy()...)
|
||||
case xml.Comment:
|
||||
lastNode.Comments = append(lastNode.Comments, tok.Copy())
|
||||
case xml.ProcInst:
|
||||
lastNode.ProcInsts = append(lastNode.ProcInsts, tok.Copy())
|
||||
}
|
||||
}
|
||||
|
||||
if len(allNodes) == 0 {
|
||||
return nil, errors.New("found no nodes")
|
||||
}
|
||||
firstNode := allNodes[0]
|
||||
trimParentNodesContentSpaces(firstNode)
|
||||
|
||||
return firstNode, nil
|
||||
}
|
||||
|
||||
func newXmlDecoder(reader io.Reader) *xml.Decoder {
|
||||
dec := xml.NewDecoder(reader)
|
||||
dec.CharsetReader = charset.NewReaderLabel
|
||||
return dec
|
||||
}
|
||||
|
||||
func trimParentNodesContentSpaces(node *xmlNode) {
|
||||
if len(node.Nodes) > 0 {
|
||||
node.Content = bytes.TrimSpace(node.Content)
|
||||
for _, childNode := range node.Nodes {
|
||||
trimParentNodesContentSpaces(childNode)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type xmlNode struct {
|
||||
XMLName xml.Name
|
||||
Comments []xml.Comment
|
||||
ProcInsts []xml.ProcInst
|
||||
XMLAttr []xml.Attr
|
||||
Content []byte
|
||||
Nodes []*xmlNode
|
||||
}
|
76
vendor/github.com/onsi/gomega/matchers/match_yaml_matcher.go
generated
vendored
Normal file
76
vendor/github.com/onsi/gomega/matchers/match_yaml_matcher.go
generated
vendored
Normal file
@ -0,0 +1,76 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
"gopkg.in/yaml.v2"
|
||||
)
|
||||
|
||||
type MatchYAMLMatcher struct {
|
||||
YAMLToMatch interface{}
|
||||
firstFailurePath []interface{}
|
||||
}
|
||||
|
||||
func (matcher *MatchYAMLMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
actualString, expectedString, err := matcher.toStrings(actual)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
var aval interface{}
|
||||
var eval interface{}
|
||||
|
||||
if err := yaml.Unmarshal([]byte(actualString), &aval); err != nil {
|
||||
return false, fmt.Errorf("Actual '%s' should be valid YAML, but it is not.\nUnderlying error:%s", actualString, err)
|
||||
}
|
||||
if err := yaml.Unmarshal([]byte(expectedString), &eval); err != nil {
|
||||
return false, fmt.Errorf("Expected '%s' should be valid YAML, but it is not.\nUnderlying error:%s", expectedString, err)
|
||||
}
|
||||
|
||||
var equal bool
|
||||
equal, matcher.firstFailurePath = deepEqual(aval, eval)
|
||||
return equal, nil
|
||||
}
|
||||
|
||||
func (matcher *MatchYAMLMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.toNormalisedStrings(actual)
|
||||
return formattedMessage(format.Message(actualString, "to match YAML of", expectedString), matcher.firstFailurePath)
|
||||
}
|
||||
|
||||
func (matcher *MatchYAMLMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
actualString, expectedString, _ := matcher.toNormalisedStrings(actual)
|
||||
return formattedMessage(format.Message(actualString, "not to match YAML of", expectedString), matcher.firstFailurePath)
|
||||
}
|
||||
|
||||
func (matcher *MatchYAMLMatcher) toNormalisedStrings(actual interface{}) (actualFormatted, expectedFormatted string, err error) {
|
||||
actualString, expectedString, err := matcher.toStrings(actual)
|
||||
return normalise(actualString), normalise(expectedString), err
|
||||
}
|
||||
|
||||
func normalise(input string) string {
|
||||
var val interface{}
|
||||
err := yaml.Unmarshal([]byte(input), &val)
|
||||
if err != nil {
|
||||
panic(err) // unreachable since Match already calls Unmarshal
|
||||
}
|
||||
output, err := yaml.Marshal(val)
|
||||
if err != nil {
|
||||
panic(err) // untested section, unreachable since we Unmarshal above
|
||||
}
|
||||
return strings.TrimSpace(string(output))
|
||||
}
|
||||
|
||||
func (matcher *MatchYAMLMatcher) toStrings(actual interface{}) (actualFormatted, expectedFormatted string, err error) {
|
||||
actualString, ok := toString(actual)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchYAMLMatcher matcher requires a string, stringer, or []byte. Got actual:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
expectedString, ok := toString(matcher.YAMLToMatch)
|
||||
if !ok {
|
||||
return "", "", fmt.Errorf("MatchYAMLMatcher matcher requires a string, stringer, or []byte. Got expected:\n%s", format.Object(matcher.YAMLToMatch, 1))
|
||||
}
|
||||
|
||||
return actualString, expectedString, nil
|
||||
}
|
30
vendor/github.com/onsi/gomega/matchers/not.go
generated
vendored
Normal file
30
vendor/github.com/onsi/gomega/matchers/not.go
generated
vendored
Normal file
@ -0,0 +1,30 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"github.com/onsi/gomega/internal/oraclematcher"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type NotMatcher struct {
|
||||
Matcher types.GomegaMatcher
|
||||
}
|
||||
|
||||
func (m *NotMatcher) Match(actual interface{}) (bool, error) {
|
||||
success, err := m.Matcher.Match(actual)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return !success, nil
|
||||
}
|
||||
|
||||
func (m *NotMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return m.Matcher.NegatedFailureMessage(actual) // works beautifully
|
||||
}
|
||||
|
||||
func (m *NotMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return m.Matcher.FailureMessage(actual) // works beautifully
|
||||
}
|
||||
|
||||
func (m *NotMatcher) MatchMayChangeInTheFuture(actual interface{}) bool {
|
||||
return oraclematcher.MatchMayChangeInTheFuture(m.Matcher, actual) // just return m.Matcher's value
|
||||
}
|
67
vendor/github.com/onsi/gomega/matchers/or.go
generated
vendored
Normal file
67
vendor/github.com/onsi/gomega/matchers/or.go
generated
vendored
Normal file
@ -0,0 +1,67 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
"github.com/onsi/gomega/internal/oraclematcher"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type OrMatcher struct {
|
||||
Matchers []types.GomegaMatcher
|
||||
|
||||
// state
|
||||
firstSuccessfulMatcher types.GomegaMatcher
|
||||
}
|
||||
|
||||
func (m *OrMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
m.firstSuccessfulMatcher = nil
|
||||
for _, matcher := range m.Matchers {
|
||||
success, err := matcher.Match(actual)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if success {
|
||||
m.firstSuccessfulMatcher = matcher
|
||||
return true, nil
|
||||
}
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (m *OrMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
// not the most beautiful list of matchers, but not bad either...
|
||||
return format.Message(actual, fmt.Sprintf("To satisfy at least one of these matchers: %s", m.Matchers))
|
||||
}
|
||||
|
||||
func (m *OrMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return m.firstSuccessfulMatcher.NegatedFailureMessage(actual)
|
||||
}
|
||||
|
||||
func (m *OrMatcher) MatchMayChangeInTheFuture(actual interface{}) bool {
|
||||
/*
|
||||
Example with 3 matchers: A, B, C
|
||||
|
||||
Match evaluates them: F, T, <?> => T
|
||||
So match is currently T, what should MatchMayChangeInTheFuture() return?
|
||||
Seems like it only depends on B, since currently B MUST change to allow the result to become F
|
||||
|
||||
Match eval: F, F, F => F
|
||||
So match is currently F, what should MatchMayChangeInTheFuture() return?
|
||||
Seems to depend on ANY of them being able to change to T.
|
||||
*/
|
||||
|
||||
if m.firstSuccessfulMatcher != nil {
|
||||
// one of the matchers succeeded.. it must be able to change in order to affect the result
|
||||
return oraclematcher.MatchMayChangeInTheFuture(m.firstSuccessfulMatcher, actual)
|
||||
} else {
|
||||
// so all matchers failed.. Any one of them changing would change the result.
|
||||
for _, matcher := range m.Matchers {
|
||||
if oraclematcher.MatchMayChangeInTheFuture(matcher, actual) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false // none of were going to change
|
||||
}
|
||||
}
|
46
vendor/github.com/onsi/gomega/matchers/panic_matcher.go
generated
vendored
Normal file
46
vendor/github.com/onsi/gomega/matchers/panic_matcher.go
generated
vendored
Normal file
@ -0,0 +1,46 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type PanicMatcher struct {
|
||||
object interface{}
|
||||
}
|
||||
|
||||
func (matcher *PanicMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if actual == nil {
|
||||
return false, fmt.Errorf("PanicMatcher expects a non-nil actual.")
|
||||
}
|
||||
|
||||
actualType := reflect.TypeOf(actual)
|
||||
if actualType.Kind() != reflect.Func {
|
||||
return false, fmt.Errorf("PanicMatcher expects a function. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
if !(actualType.NumIn() == 0 && actualType.NumOut() == 0) {
|
||||
return false, fmt.Errorf("PanicMatcher expects a function with no arguments and no return value. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
success = false
|
||||
defer func() {
|
||||
if e := recover(); e != nil {
|
||||
matcher.object = e
|
||||
success = true
|
||||
}
|
||||
}()
|
||||
|
||||
reflect.ValueOf(actual).Call([]reflect.Value{})
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (matcher *PanicMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, "to panic")
|
||||
}
|
||||
|
||||
func (matcher *PanicMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return format.Message(actual, fmt.Sprintf("not to panic, but panicked with\n%s", format.Object(matcher.object, 1)))
|
||||
}
|
128
vendor/github.com/onsi/gomega/matchers/receive_matcher.go
generated
vendored
Normal file
128
vendor/github.com/onsi/gomega/matchers/receive_matcher.go
generated
vendored
Normal file
@ -0,0 +1,128 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type ReceiveMatcher struct {
|
||||
Arg interface{}
|
||||
receivedValue reflect.Value
|
||||
channelClosed bool
|
||||
}
|
||||
|
||||
func (matcher *ReceiveMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
if !isChan(actual) {
|
||||
return false, fmt.Errorf("ReceiveMatcher expects a channel. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
channelType := reflect.TypeOf(actual)
|
||||
channelValue := reflect.ValueOf(actual)
|
||||
|
||||
if channelType.ChanDir() == reflect.SendDir {
|
||||
return false, fmt.Errorf("ReceiveMatcher matcher cannot be passed a send-only channel. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
var subMatcher omegaMatcher
|
||||
var hasSubMatcher bool
|
||||
|
||||
if matcher.Arg != nil {
|
||||
subMatcher, hasSubMatcher = (matcher.Arg).(omegaMatcher)
|
||||
if !hasSubMatcher {
|
||||
argType := reflect.TypeOf(matcher.Arg)
|
||||
if argType.Kind() != reflect.Ptr {
|
||||
return false, fmt.Errorf("Cannot assign a value from the channel:\n%s\nTo:\n%s\nYou need to pass a pointer!", format.Object(actual, 1), format.Object(matcher.Arg, 1))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
winnerIndex, value, open := reflect.Select([]reflect.SelectCase{
|
||||
{Dir: reflect.SelectRecv, Chan: channelValue},
|
||||
{Dir: reflect.SelectDefault},
|
||||
})
|
||||
|
||||
var closed bool
|
||||
var didReceive bool
|
||||
if winnerIndex == 0 {
|
||||
closed = !open
|
||||
didReceive = open
|
||||
}
|
||||
matcher.channelClosed = closed
|
||||
|
||||
if closed {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if hasSubMatcher {
|
||||
if didReceive {
|
||||
matcher.receivedValue = value
|
||||
return subMatcher.Match(matcher.receivedValue.Interface())
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if didReceive {
|
||||
if matcher.Arg != nil {
|
||||
outValue := reflect.ValueOf(matcher.Arg)
|
||||
|
||||
if value.Type().AssignableTo(outValue.Elem().Type()) {
|
||||
outValue.Elem().Set(value)
|
||||
return true, nil
|
||||
}
|
||||
if value.Type().Kind() == reflect.Interface && value.Elem().Type().AssignableTo(outValue.Elem().Type()) {
|
||||
outValue.Elem().Set(value.Elem())
|
||||
return true, nil
|
||||
} else {
|
||||
return false, fmt.Errorf("Cannot assign a value from the channel:\n%s\nType:\n%s\nTo:\n%s", format.Object(actual, 1), format.Object(value.Interface(), 1), format.Object(matcher.Arg, 1))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return true, nil
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (matcher *ReceiveMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
subMatcher, hasSubMatcher := (matcher.Arg).(omegaMatcher)
|
||||
|
||||
closedAddendum := ""
|
||||
if matcher.channelClosed {
|
||||
closedAddendum = " The channel is closed."
|
||||
}
|
||||
|
||||
if hasSubMatcher {
|
||||
if matcher.receivedValue.IsValid() {
|
||||
return subMatcher.FailureMessage(matcher.receivedValue.Interface())
|
||||
}
|
||||
return "When passed a matcher, ReceiveMatcher's channel *must* receive something."
|
||||
}
|
||||
return format.Message(actual, "to receive something."+closedAddendum)
|
||||
}
|
||||
|
||||
func (matcher *ReceiveMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
subMatcher, hasSubMatcher := (matcher.Arg).(omegaMatcher)
|
||||
|
||||
closedAddendum := ""
|
||||
if matcher.channelClosed {
|
||||
closedAddendum = " The channel is closed."
|
||||
}
|
||||
|
||||
if hasSubMatcher {
|
||||
if matcher.receivedValue.IsValid() {
|
||||
return subMatcher.NegatedFailureMessage(matcher.receivedValue.Interface())
|
||||
}
|
||||
return "When passed a matcher, ReceiveMatcher's channel *must* receive something."
|
||||
}
|
||||
return format.Message(actual, "not to receive anything."+closedAddendum)
|
||||
}
|
||||
|
||||
func (matcher *ReceiveMatcher) MatchMayChangeInTheFuture(actual interface{}) bool {
|
||||
if !isChan(actual) {
|
||||
return false
|
||||
}
|
||||
|
||||
return !matcher.channelClosed
|
||||
}
|
92
vendor/github.com/onsi/gomega/matchers/semi_structured_data_support.go
generated
vendored
Normal file
92
vendor/github.com/onsi/gomega/matchers/semi_structured_data_support.go
generated
vendored
Normal file
@ -0,0 +1,92 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func formattedMessage(comparisonMessage string, failurePath []interface{}) string {
|
||||
var diffMessage string
|
||||
if len(failurePath) == 0 {
|
||||
diffMessage = ""
|
||||
} else {
|
||||
diffMessage = fmt.Sprintf("\n\nfirst mismatched key: %s", formattedFailurePath(failurePath))
|
||||
}
|
||||
return fmt.Sprintf("%s%s", comparisonMessage, diffMessage)
|
||||
}
|
||||
|
||||
func formattedFailurePath(failurePath []interface{}) string {
|
||||
formattedPaths := []string{}
|
||||
for i := len(failurePath) - 1; i >= 0; i-- {
|
||||
switch p := failurePath[i].(type) {
|
||||
case int:
|
||||
formattedPaths = append(formattedPaths, fmt.Sprintf(`[%d]`, p))
|
||||
default:
|
||||
if i != len(failurePath)-1 {
|
||||
formattedPaths = append(formattedPaths, ".")
|
||||
}
|
||||
formattedPaths = append(formattedPaths, fmt.Sprintf(`"%s"`, p))
|
||||
}
|
||||
}
|
||||
return strings.Join(formattedPaths, "")
|
||||
}
|
||||
|
||||
func deepEqual(a interface{}, b interface{}) (bool, []interface{}) {
|
||||
var errorPath []interface{}
|
||||
if reflect.TypeOf(a) != reflect.TypeOf(b) {
|
||||
return false, errorPath
|
||||
}
|
||||
|
||||
switch a.(type) {
|
||||
case []interface{}:
|
||||
if len(a.([]interface{})) != len(b.([]interface{})) {
|
||||
return false, errorPath
|
||||
}
|
||||
|
||||
for i, v := range a.([]interface{}) {
|
||||
elementEqual, keyPath := deepEqual(v, b.([]interface{})[i])
|
||||
if !elementEqual {
|
||||
return false, append(keyPath, i)
|
||||
}
|
||||
}
|
||||
return true, errorPath
|
||||
|
||||
case map[interface{}]interface{}:
|
||||
if len(a.(map[interface{}]interface{})) != len(b.(map[interface{}]interface{})) {
|
||||
return false, errorPath
|
||||
}
|
||||
|
||||
for k, v1 := range a.(map[interface{}]interface{}) {
|
||||
v2, ok := b.(map[interface{}]interface{})[k]
|
||||
if !ok {
|
||||
return false, errorPath
|
||||
}
|
||||
elementEqual, keyPath := deepEqual(v1, v2)
|
||||
if !elementEqual {
|
||||
return false, append(keyPath, k)
|
||||
}
|
||||
}
|
||||
return true, errorPath
|
||||
|
||||
case map[string]interface{}:
|
||||
if len(a.(map[string]interface{})) != len(b.(map[string]interface{})) {
|
||||
return false, errorPath
|
||||
}
|
||||
|
||||
for k, v1 := range a.(map[string]interface{}) {
|
||||
v2, ok := b.(map[string]interface{})[k]
|
||||
if !ok {
|
||||
return false, errorPath
|
||||
}
|
||||
elementEqual, keyPath := deepEqual(v1, v2)
|
||||
if !elementEqual {
|
||||
return false, append(keyPath, k)
|
||||
}
|
||||
}
|
||||
return true, errorPath
|
||||
|
||||
default:
|
||||
return a == b, errorPath
|
||||
}
|
||||
}
|
33
vendor/github.com/onsi/gomega/matchers/succeed_matcher.go
generated
vendored
Normal file
33
vendor/github.com/onsi/gomega/matchers/succeed_matcher.go
generated
vendored
Normal file
@ -0,0 +1,33 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/onsi/gomega/format"
|
||||
)
|
||||
|
||||
type SucceedMatcher struct {
|
||||
}
|
||||
|
||||
func (matcher *SucceedMatcher) Match(actual interface{}) (success bool, err error) {
|
||||
// is purely nil?
|
||||
if actual == nil {
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// must be an 'error' type
|
||||
if !isError(actual) {
|
||||
return false, fmt.Errorf("Expected an error-type. Got:\n%s", format.Object(actual, 1))
|
||||
}
|
||||
|
||||
// must be nil (or a pointer to a nil)
|
||||
return isNil(actual), nil
|
||||
}
|
||||
|
||||
func (matcher *SucceedMatcher) FailureMessage(actual interface{}) (message string) {
|
||||
return fmt.Sprintf("Expected success, but got an error:\n%s\n%s", format.Object(actual, 1), format.IndentString(actual.(error).Error(), 1))
|
||||
}
|
||||
|
||||
func (matcher *SucceedMatcher) NegatedFailureMessage(actual interface{}) (message string) {
|
||||
return "Expected failure, but got no error."
|
||||
}
|
41
vendor/github.com/onsi/gomega/matchers/support/goraph/bipartitegraph/bipartitegraph.go
generated
vendored
Normal file
41
vendor/github.com/onsi/gomega/matchers/support/goraph/bipartitegraph/bipartitegraph.go
generated
vendored
Normal file
@ -0,0 +1,41 @@
|
||||
package bipartitegraph
|
||||
|
||||
import "errors"
|
||||
import "fmt"
|
||||
|
||||
import . "github.com/onsi/gomega/matchers/support/goraph/node"
|
||||
import . "github.com/onsi/gomega/matchers/support/goraph/edge"
|
||||
|
||||
type BipartiteGraph struct {
|
||||
Left NodeOrderedSet
|
||||
Right NodeOrderedSet
|
||||
Edges EdgeSet
|
||||
}
|
||||
|
||||
func NewBipartiteGraph(leftValues, rightValues []interface{}, neighbours func(interface{}, interface{}) (bool, error)) (*BipartiteGraph, error) {
|
||||
left := NodeOrderedSet{}
|
||||
for i := range leftValues {
|
||||
left = append(left, Node{Id: i})
|
||||
}
|
||||
|
||||
right := NodeOrderedSet{}
|
||||
for j := range rightValues {
|
||||
right = append(right, Node{Id: j + len(left)})
|
||||
}
|
||||
|
||||
edges := EdgeSet{}
|
||||
for i, leftValue := range leftValues {
|
||||
for j, rightValue := range rightValues {
|
||||
neighbours, err := neighbours(leftValue, rightValue)
|
||||
if err != nil {
|
||||
return nil, errors.New(fmt.Sprintf("error determining adjacency for %v and %v: %s", leftValue, rightValue, err.Error()))
|
||||
}
|
||||
|
||||
if neighbours {
|
||||
edges = append(edges, Edge{Node1: left[i], Node2: right[j]})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return &BipartiteGraph{left, right, edges}, nil
|
||||
}
|
159
vendor/github.com/onsi/gomega/matchers/support/goraph/bipartitegraph/bipartitegraphmatching.go
generated
vendored
Normal file
159
vendor/github.com/onsi/gomega/matchers/support/goraph/bipartitegraph/bipartitegraphmatching.go
generated
vendored
Normal file
@ -0,0 +1,159 @@
|
||||
package bipartitegraph
|
||||
|
||||
import . "github.com/onsi/gomega/matchers/support/goraph/node"
|
||||
import . "github.com/onsi/gomega/matchers/support/goraph/edge"
|
||||
import "github.com/onsi/gomega/matchers/support/goraph/util"
|
||||
|
||||
func (bg *BipartiteGraph) LargestMatching() (matching EdgeSet) {
|
||||
paths := bg.maximalDisjointSLAPCollection(matching)
|
||||
|
||||
for len(paths) > 0 {
|
||||
for _, path := range paths {
|
||||
matching = matching.SymmetricDifference(path)
|
||||
}
|
||||
paths = bg.maximalDisjointSLAPCollection(matching)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (bg *BipartiteGraph) maximalDisjointSLAPCollection(matching EdgeSet) (result []EdgeSet) {
|
||||
guideLayers := bg.createSLAPGuideLayers(matching)
|
||||
if len(guideLayers) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
used := make(map[Node]bool)
|
||||
|
||||
for _, u := range guideLayers[len(guideLayers)-1] {
|
||||
slap, found := bg.findDisjointSLAP(u, matching, guideLayers, used)
|
||||
if found {
|
||||
for _, edge := range slap {
|
||||
used[edge.Node1] = true
|
||||
used[edge.Node2] = true
|
||||
}
|
||||
result = append(result, slap)
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (bg *BipartiteGraph) findDisjointSLAP(
|
||||
start Node,
|
||||
matching EdgeSet,
|
||||
guideLayers []NodeOrderedSet,
|
||||
used map[Node]bool,
|
||||
) ([]Edge, bool) {
|
||||
return bg.findDisjointSLAPHelper(start, EdgeSet{}, len(guideLayers)-1, matching, guideLayers, used)
|
||||
}
|
||||
|
||||
func (bg *BipartiteGraph) findDisjointSLAPHelper(
|
||||
currentNode Node,
|
||||
currentSLAP EdgeSet,
|
||||
currentLevel int,
|
||||
matching EdgeSet,
|
||||
guideLayers []NodeOrderedSet,
|
||||
used map[Node]bool,
|
||||
) (EdgeSet, bool) {
|
||||
used[currentNode] = true
|
||||
|
||||
if currentLevel == 0 {
|
||||
return currentSLAP, true
|
||||
}
|
||||
|
||||
for _, nextNode := range guideLayers[currentLevel-1] {
|
||||
if used[nextNode] {
|
||||
continue
|
||||
}
|
||||
|
||||
edge, found := bg.Edges.FindByNodes(currentNode, nextNode)
|
||||
if !found {
|
||||
continue
|
||||
}
|
||||
|
||||
if matching.Contains(edge) == util.Odd(currentLevel) {
|
||||
continue
|
||||
}
|
||||
|
||||
currentSLAP = append(currentSLAP, edge)
|
||||
slap, found := bg.findDisjointSLAPHelper(nextNode, currentSLAP, currentLevel-1, matching, guideLayers, used)
|
||||
if found {
|
||||
return slap, true
|
||||
}
|
||||
currentSLAP = currentSLAP[:len(currentSLAP)-1]
|
||||
}
|
||||
|
||||
used[currentNode] = false
|
||||
return nil, false
|
||||
}
|
||||
|
||||
func (bg *BipartiteGraph) createSLAPGuideLayers(matching EdgeSet) (guideLayers []NodeOrderedSet) {
|
||||
used := make(map[Node]bool)
|
||||
currentLayer := NodeOrderedSet{}
|
||||
|
||||
for _, node := range bg.Left {
|
||||
if matching.Free(node) {
|
||||
used[node] = true
|
||||
currentLayer = append(currentLayer, node)
|
||||
}
|
||||
}
|
||||
|
||||
if len(currentLayer) == 0 {
|
||||
return []NodeOrderedSet{}
|
||||
}
|
||||
guideLayers = append(guideLayers, currentLayer)
|
||||
|
||||
done := false
|
||||
|
||||
for !done {
|
||||
lastLayer := currentLayer
|
||||
currentLayer = NodeOrderedSet{}
|
||||
|
||||
if util.Odd(len(guideLayers)) {
|
||||
for _, leftNode := range lastLayer {
|
||||
for _, rightNode := range bg.Right {
|
||||
if used[rightNode] {
|
||||
continue
|
||||
}
|
||||
|
||||
edge, found := bg.Edges.FindByNodes(leftNode, rightNode)
|
||||
if !found || matching.Contains(edge) {
|
||||
continue
|
||||
}
|
||||
|
||||
currentLayer = append(currentLayer, rightNode)
|
||||
used[rightNode] = true
|
||||
|
||||
if matching.Free(rightNode) {
|
||||
done = true
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for _, rightNode := range lastLayer {
|
||||
for _, leftNode := range bg.Left {
|
||||
if used[leftNode] {
|
||||
continue
|
||||
}
|
||||
|
||||
edge, found := bg.Edges.FindByNodes(leftNode, rightNode)
|
||||
if !found || !matching.Contains(edge) {
|
||||
continue
|
||||
}
|
||||
|
||||
currentLayer = append(currentLayer, leftNode)
|
||||
used[leftNode] = true
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if len(currentLayer) == 0 {
|
||||
return []NodeOrderedSet{}
|
||||
}
|
||||
guideLayers = append(guideLayers, currentLayer)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
61
vendor/github.com/onsi/gomega/matchers/support/goraph/edge/edge.go
generated
vendored
Normal file
61
vendor/github.com/onsi/gomega/matchers/support/goraph/edge/edge.go
generated
vendored
Normal file
@ -0,0 +1,61 @@
|
||||
package edge
|
||||
|
||||
import . "github.com/onsi/gomega/matchers/support/goraph/node"
|
||||
|
||||
type Edge struct {
|
||||
Node1 Node
|
||||
Node2 Node
|
||||
}
|
||||
|
||||
type EdgeSet []Edge
|
||||
|
||||
func (ec EdgeSet) Free(node Node) bool {
|
||||
for _, e := range ec {
|
||||
if e.Node1 == node || e.Node2 == node {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (ec EdgeSet) Contains(edge Edge) bool {
|
||||
for _, e := range ec {
|
||||
if e == edge {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func (ec EdgeSet) FindByNodes(node1, node2 Node) (Edge, bool) {
|
||||
for _, e := range ec {
|
||||
if (e.Node1 == node1 && e.Node2 == node2) || (e.Node1 == node2 && e.Node2 == node1) {
|
||||
return e, true
|
||||
}
|
||||
}
|
||||
|
||||
return Edge{}, false
|
||||
}
|
||||
|
||||
func (ec EdgeSet) SymmetricDifference(ec2 EdgeSet) EdgeSet {
|
||||
edgesToInclude := make(map[Edge]bool)
|
||||
|
||||
for _, e := range ec {
|
||||
edgesToInclude[e] = true
|
||||
}
|
||||
|
||||
for _, e := range ec2 {
|
||||
edgesToInclude[e] = !edgesToInclude[e]
|
||||
}
|
||||
|
||||
result := EdgeSet{}
|
||||
for e, include := range edgesToInclude {
|
||||
if include {
|
||||
result = append(result, e)
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
7
vendor/github.com/onsi/gomega/matchers/support/goraph/node/node.go
generated
vendored
Normal file
7
vendor/github.com/onsi/gomega/matchers/support/goraph/node/node.go
generated
vendored
Normal file
@ -0,0 +1,7 @@
|
||||
package node
|
||||
|
||||
type Node struct {
|
||||
Id int
|
||||
}
|
||||
|
||||
type NodeOrderedSet []Node
|
7
vendor/github.com/onsi/gomega/matchers/support/goraph/util/util.go
generated
vendored
Normal file
7
vendor/github.com/onsi/gomega/matchers/support/goraph/util/util.go
generated
vendored
Normal file
@ -0,0 +1,7 @@
|
||||
package util
|
||||
|
||||
import "math"
|
||||
|
||||
func Odd(n int) bool {
|
||||
return math.Mod(float64(n), 2.0) == 1.0
|
||||
}
|
179
vendor/github.com/onsi/gomega/matchers/type_support.go
generated
vendored
Normal file
179
vendor/github.com/onsi/gomega/matchers/type_support.go
generated
vendored
Normal file
@ -0,0 +1,179 @@
|
||||
/*
|
||||
Gomega matchers
|
||||
|
||||
This package implements the Gomega matchers and does not typically need to be imported.
|
||||
See the docs for Gomega for documentation on the matchers
|
||||
|
||||
http://onsi.github.io/gomega/
|
||||
*/
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"reflect"
|
||||
)
|
||||
|
||||
type omegaMatcher interface {
|
||||
Match(actual interface{}) (success bool, err error)
|
||||
FailureMessage(actual interface{}) (message string)
|
||||
NegatedFailureMessage(actual interface{}) (message string)
|
||||
}
|
||||
|
||||
func isBool(a interface{}) bool {
|
||||
return reflect.TypeOf(a).Kind() == reflect.Bool
|
||||
}
|
||||
|
||||
func isNumber(a interface{}) bool {
|
||||
if a == nil {
|
||||
return false
|
||||
}
|
||||
kind := reflect.TypeOf(a).Kind()
|
||||
return reflect.Int <= kind && kind <= reflect.Float64
|
||||
}
|
||||
|
||||
func isInteger(a interface{}) bool {
|
||||
kind := reflect.TypeOf(a).Kind()
|
||||
return reflect.Int <= kind && kind <= reflect.Int64
|
||||
}
|
||||
|
||||
func isUnsignedInteger(a interface{}) bool {
|
||||
kind := reflect.TypeOf(a).Kind()
|
||||
return reflect.Uint <= kind && kind <= reflect.Uint64
|
||||
}
|
||||
|
||||
func isFloat(a interface{}) bool {
|
||||
kind := reflect.TypeOf(a).Kind()
|
||||
return reflect.Float32 <= kind && kind <= reflect.Float64
|
||||
}
|
||||
|
||||
func toInteger(a interface{}) int64 {
|
||||
if isInteger(a) {
|
||||
return reflect.ValueOf(a).Int()
|
||||
} else if isUnsignedInteger(a) {
|
||||
return int64(reflect.ValueOf(a).Uint())
|
||||
} else if isFloat(a) {
|
||||
return int64(reflect.ValueOf(a).Float())
|
||||
}
|
||||
panic(fmt.Sprintf("Expected a number! Got <%T> %#v", a, a))
|
||||
}
|
||||
|
||||
func toUnsignedInteger(a interface{}) uint64 {
|
||||
if isInteger(a) {
|
||||
return uint64(reflect.ValueOf(a).Int())
|
||||
} else if isUnsignedInteger(a) {
|
||||
return reflect.ValueOf(a).Uint()
|
||||
} else if isFloat(a) {
|
||||
return uint64(reflect.ValueOf(a).Float())
|
||||
}
|
||||
panic(fmt.Sprintf("Expected a number! Got <%T> %#v", a, a))
|
||||
}
|
||||
|
||||
func toFloat(a interface{}) float64 {
|
||||
if isInteger(a) {
|
||||
return float64(reflect.ValueOf(a).Int())
|
||||
} else if isUnsignedInteger(a) {
|
||||
return float64(reflect.ValueOf(a).Uint())
|
||||
} else if isFloat(a) {
|
||||
return reflect.ValueOf(a).Float()
|
||||
}
|
||||
panic(fmt.Sprintf("Expected a number! Got <%T> %#v", a, a))
|
||||
}
|
||||
|
||||
func isError(a interface{}) bool {
|
||||
_, ok := a.(error)
|
||||
return ok
|
||||
}
|
||||
|
||||
func isChan(a interface{}) bool {
|
||||
if isNil(a) {
|
||||
return false
|
||||
}
|
||||
return reflect.TypeOf(a).Kind() == reflect.Chan
|
||||
}
|
||||
|
||||
func isMap(a interface{}) bool {
|
||||
if a == nil {
|
||||
return false
|
||||
}
|
||||
return reflect.TypeOf(a).Kind() == reflect.Map
|
||||
}
|
||||
|
||||
func isArrayOrSlice(a interface{}) bool {
|
||||
if a == nil {
|
||||
return false
|
||||
}
|
||||
switch reflect.TypeOf(a).Kind() {
|
||||
case reflect.Array, reflect.Slice:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func isString(a interface{}) bool {
|
||||
if a == nil {
|
||||
return false
|
||||
}
|
||||
return reflect.TypeOf(a).Kind() == reflect.String
|
||||
}
|
||||
|
||||
func toString(a interface{}) (string, bool) {
|
||||
aString, isString := a.(string)
|
||||
if isString {
|
||||
return aString, true
|
||||
}
|
||||
|
||||
aBytes, isBytes := a.([]byte)
|
||||
if isBytes {
|
||||
return string(aBytes), true
|
||||
}
|
||||
|
||||
aStringer, isStringer := a.(fmt.Stringer)
|
||||
if isStringer {
|
||||
return aStringer.String(), true
|
||||
}
|
||||
|
||||
aJSONRawMessage, isJSONRawMessage := a.(json.RawMessage)
|
||||
if isJSONRawMessage {
|
||||
return string(aJSONRawMessage), true
|
||||
}
|
||||
|
||||
return "", false
|
||||
}
|
||||
|
||||
func lengthOf(a interface{}) (int, bool) {
|
||||
if a == nil {
|
||||
return 0, false
|
||||
}
|
||||
switch reflect.TypeOf(a).Kind() {
|
||||
case reflect.Map, reflect.Array, reflect.String, reflect.Chan, reflect.Slice:
|
||||
return reflect.ValueOf(a).Len(), true
|
||||
default:
|
||||
return 0, false
|
||||
}
|
||||
}
|
||||
func capOf(a interface{}) (int, bool) {
|
||||
if a == nil {
|
||||
return 0, false
|
||||
}
|
||||
switch reflect.TypeOf(a).Kind() {
|
||||
case reflect.Array, reflect.Chan, reflect.Slice:
|
||||
return reflect.ValueOf(a).Cap(), true
|
||||
default:
|
||||
return 0, false
|
||||
}
|
||||
}
|
||||
|
||||
func isNil(a interface{}) bool {
|
||||
if a == nil {
|
||||
return true
|
||||
}
|
||||
|
||||
switch reflect.TypeOf(a).Kind() {
|
||||
case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice:
|
||||
return reflect.ValueOf(a).IsNil()
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
72
vendor/github.com/onsi/gomega/matchers/with_transform.go
generated
vendored
Normal file
72
vendor/github.com/onsi/gomega/matchers/with_transform.go
generated
vendored
Normal file
@ -0,0 +1,72 @@
|
||||
package matchers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"github.com/onsi/gomega/internal/oraclematcher"
|
||||
"github.com/onsi/gomega/types"
|
||||
)
|
||||
|
||||
type WithTransformMatcher struct {
|
||||
// input
|
||||
Transform interface{} // must be a function of one parameter that returns one value
|
||||
Matcher types.GomegaMatcher
|
||||
|
||||
// cached value
|
||||
transformArgType reflect.Type
|
||||
|
||||
// state
|
||||
transformedValue interface{}
|
||||
}
|
||||
|
||||
func NewWithTransformMatcher(transform interface{}, matcher types.GomegaMatcher) *WithTransformMatcher {
|
||||
if transform == nil {
|
||||
panic("transform function cannot be nil")
|
||||
}
|
||||
txType := reflect.TypeOf(transform)
|
||||
if txType.NumIn() != 1 {
|
||||
panic("transform function must have 1 argument")
|
||||
}
|
||||
if txType.NumOut() != 1 {
|
||||
panic("transform function must have 1 return value")
|
||||
}
|
||||
|
||||
return &WithTransformMatcher{
|
||||
Transform: transform,
|
||||
Matcher: matcher,
|
||||
transformArgType: reflect.TypeOf(transform).In(0),
|
||||
}
|
||||
}
|
||||
|
||||
func (m *WithTransformMatcher) Match(actual interface{}) (bool, error) {
|
||||
// return error if actual's type is incompatible with Transform function's argument type
|
||||
actualType := reflect.TypeOf(actual)
|
||||
if !actualType.AssignableTo(m.transformArgType) {
|
||||
return false, fmt.Errorf("Transform function expects '%s' but we have '%s'", m.transformArgType, actualType)
|
||||
}
|
||||
|
||||
// call the Transform function with `actual`
|
||||
fn := reflect.ValueOf(m.Transform)
|
||||
result := fn.Call([]reflect.Value{reflect.ValueOf(actual)})
|
||||
m.transformedValue = result[0].Interface() // expect exactly one value
|
||||
|
||||
return m.Matcher.Match(m.transformedValue)
|
||||
}
|
||||
|
||||
func (m *WithTransformMatcher) FailureMessage(_ interface{}) (message string) {
|
||||
return m.Matcher.FailureMessage(m.transformedValue)
|
||||
}
|
||||
|
||||
func (m *WithTransformMatcher) NegatedFailureMessage(_ interface{}) (message string) {
|
||||
return m.Matcher.NegatedFailureMessage(m.transformedValue)
|
||||
}
|
||||
|
||||
func (m *WithTransformMatcher) MatchMayChangeInTheFuture(_ interface{}) bool {
|
||||
// TODO: Maybe this should always just return true? (Only an issue for non-deterministic transformers.)
|
||||
//
|
||||
// Querying the next matcher is fine if the transformer always will return the same value.
|
||||
// But if the transformer is non-deterministic and returns a different value each time, then there
|
||||
// is no point in querying the next matcher, since it can only comment on the last transformed value.
|
||||
return oraclematcher.MatchMayChangeInTheFuture(m.Matcher, m.transformedValue)
|
||||
}
|
26
vendor/github.com/onsi/gomega/types/types.go
generated
vendored
Normal file
26
vendor/github.com/onsi/gomega/types/types.go
generated
vendored
Normal file
@ -0,0 +1,26 @@
|
||||
package types
|
||||
|
||||
type TWithHelper interface {
|
||||
Helper()
|
||||
}
|
||||
|
||||
type GomegaFailHandler func(message string, callerSkip ...int)
|
||||
|
||||
type GomegaFailWrapper struct {
|
||||
Fail GomegaFailHandler
|
||||
TWithHelper TWithHelper
|
||||
}
|
||||
|
||||
//A simple *testing.T interface wrapper
|
||||
type GomegaTestingT interface {
|
||||
Fatalf(format string, args ...interface{})
|
||||
}
|
||||
|
||||
//All Gomega matchers must implement the GomegaMatcher interface
|
||||
//
|
||||
//For details on writing custom matchers, check out: http://onsi.github.io/gomega/#adding-your-own-matchers
|
||||
type GomegaMatcher interface {
|
||||
Match(actual interface{}) (success bool, err error)
|
||||
FailureMessage(actual interface{}) (message string)
|
||||
NegatedFailureMessage(actual interface{}) (message string)
|
||||
}
|
Loading…
Reference in New Issue
Block a user
I keep confusing the two concepts of subscription that we have within the statediff pkg:
I wonder if there's a way to distinguish these a bit more? Perhaps I am getting it confused because it's new, and I need to sit with the code a bit more. So this may be a non-issue 🙃
I'm definitely not opposed to making this distinction more clear, the issue is that the rpc connection subscription method needs to be named
Subscribe
to work with this rpcClient method.