Relay all block data + statediffs over full node websocket subscription #8

Closed
i-norden wants to merge 17 commits from rpc_statediffs_at_head into staging
87 changed files with 6674 additions and 33 deletions

View File

@ -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
}

View File

@ -142,6 +142,7 @@ var (
utils.GpoPercentileFlag,
utils.EWASMInterpreterFlag,
utils.EVMInterpreterFlag,
utils.StateDiffFlag,
configFileFlag,
}

View File

@ -245,6 +245,12 @@ var AppHelpFlagGroups = []flagGroup{
utils.MinerLegacyExtraDataFlag,
},
},
{
Name: "STATE DIFF",
Flags: []cli.Flag{
utils.StateDiffFlag,
},
},
{
Name: "MISC",
},

View File

@ -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(&ethServ)
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")

View File

@ -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.

View File

@ -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
}

View File

@ -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)

View File

@ -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
View 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) {
elizabethengelman commented 2019-04-18 19:44:54 +00:00 (Migrated from github.com)
Review

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 keep confusing the two concepts of subscription that we have within the statediff pkg: - [subscription to the blockchain's chain events](https://github.com/vulcanize/go-ethereum/blob/edf001e1d2296951e7e592c55e66ce074bd62807/statediff/service.go#L113) - [rpc connection subscription](https://github.com/vulcanize/go-ethereum/blob/edf001e1d2296951e7e592c55e66ce074bd62807/statediff/api.go#L53) 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 🙃
Review

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.

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](https://github.com/vulcanize/go-ethereum/blob/rpc_statediffs_at_head/rpc/client.go#L405).
// 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():
elizabethengelman commented 2019-04-18 19:46:28 +00:00 (Migrated from github.com)
Review

It looks like this Closed() method is deprecated in favor of an error channel

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
Review

Nice catch! So that means we can just drop this notifier.Closed() and the rpcSub.Err() should take care of everything?

Nice catch! So that means we can just drop this `notifier.Closed()` and the `rpcSub.Err()` should take care of everything?
elizabethengelman commented 2019-04-22 21:01:21 +00:00 (Migrated from github.com)
Review

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
View 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
View 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
View 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
View 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
View 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())
}
}

View 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)
}
}

View 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
}

View 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
}

View 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
}

View 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
}

View 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")
}
}

View 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
View 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
}
*/

View File

@ -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]
}

View File

@ -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)
}
}

View File

@ -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
View 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
View 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
View 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
View 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
View 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
View 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
View 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
View 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
View 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
View 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))
}

View 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, ""
}

View 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, ""
}

View 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)
}

View 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
View 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
View 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)
}

View 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))
}

View 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] }

View 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"))
}

View 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"))
}

View 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"))
}

View 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")
}

View 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")
}

View 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)
}

View 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")
}

View 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)
}

View 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")
}

View 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
}

View 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
}

View 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
}

View 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")
}

View 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
View 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)
}

View 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)
}

View 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())
}

View 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)
}

View 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)
}

View 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)
}
}

View 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)
}

View 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)
}

View 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")
}

View 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())
}

View 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())
}

View 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)
}

View 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
}

View 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
}

View 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
}

View 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
View 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
View 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
}
}

View 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)))
}

View 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
}

View 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
}
}

View 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."
}

View 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
}

View 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
}

View 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
}

View File

@ -0,0 +1,7 @@
package node
type Node struct {
Id int
}
type NodeOrderedSet []Node

View 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
View 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
}

View 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
View 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)
}