1 Star 1 Fork 0

Hyperledger Fabric 国密 / fabric

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
克隆/下载
lockbased_txmgr.go 22.09 KB
一键复制 编辑 原始数据 按行查看 历史
Jtyoui 提交于 2021-07-22 15:59 . 国密
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572
/*
Copyright IBM Corp. All Rights Reserved.
SPDX-License-Identifier: Apache-2.0
*/
package lockbasedtxmgr
import (
"bytes"
"sync"
"github.com/golang/protobuf/proto"
"gitee.com/hyperledger-fabric-gm/fabric/common/flogging"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/bookkeeping"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/privacyenabledstate"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/pvtstatepurgemgmt"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/queryutil"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/txmgr"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/validator"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/validator/valimpl"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/kvledger/txmgmt/version"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/pvtdatapolicy"
"gitee.com/hyperledger-fabric-gm/fabric/core/ledger/util"
"gitee.com/hyperledger-fabric-gm/fabric/protos/common"
"gitee.com/hyperledger-fabric-gm/fabric/protos/ledger/rwset"
"gitee.com/hyperledger-fabric-gm/fabric/protos/ledger/rwset/kvrwset"
)
var logger = flogging.MustGetLogger("lockbasedtxmgr")
// LockBasedTxMgr a simple implementation of interface `txmgmt.TxMgr`.
// This implementation uses a read-write lock to prevent conflicts between transaction simulation and committing
type LockBasedTxMgr struct {
ledgerid string
db privacyenabledstate.DB
pvtdataPurgeMgr *pvtdataPurgeMgr
validator validator.Validator
stateListeners []ledger.StateListener
ccInfoProvider ledger.DeployedChaincodeInfoProvider
commitRWLock sync.RWMutex
oldBlockCommit sync.Mutex
current *current
}
type current struct {
block *common.Block
batch *privacyenabledstate.UpdateBatch
listeners []ledger.StateListener
}
func (c *current) blockNum() uint64 {
return c.block.Header.Number
}
func (c *current) maxTxNumber() uint64 {
return uint64(len(c.block.Data.Data)) - 1
}
// NewLockBasedTxMgr constructs a new instance of NewLockBasedTxMgr
func NewLockBasedTxMgr(ledgerid string, db privacyenabledstate.DB, stateListeners []ledger.StateListener,
btlPolicy pvtdatapolicy.BTLPolicy, bookkeepingProvider bookkeeping.Provider, ccInfoProvider ledger.DeployedChaincodeInfoProvider) (*LockBasedTxMgr, error) {
db.Open()
txmgr := &LockBasedTxMgr{
ledgerid: ledgerid,
db: db,
stateListeners: stateListeners,
ccInfoProvider: ccInfoProvider,
}
pvtstatePurgeMgr, err := pvtstatepurgemgmt.InstantiatePurgeMgr(ledgerid, db, btlPolicy, bookkeepingProvider)
if err != nil {
return nil, err
}
txmgr.pvtdataPurgeMgr = &pvtdataPurgeMgr{pvtstatePurgeMgr, false}
txmgr.validator = valimpl.NewStatebasedValidator(txmgr, db)
return txmgr, nil
}
// GetLastSavepoint returns the block num recorded in savepoint,
// returns 0 if NO savepoint is found
func (txmgr *LockBasedTxMgr) GetLastSavepoint() (*version.Height, error) {
return txmgr.db.GetLatestSavePoint()
}
// NewQueryExecutor implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) NewQueryExecutor(txid string) (ledger.QueryExecutor, error) {
qe := newQueryExecutor(txmgr, txid)
txmgr.commitRWLock.RLock()
return qe, nil
}
// NewTxSimulator implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) NewTxSimulator(txid string) (ledger.TxSimulator, error) {
logger.Debugf("constructing new tx simulator")
s, err := newLockBasedTxSimulator(txmgr, txid)
if err != nil {
return nil, err
}
txmgr.commitRWLock.RLock()
return s, nil
}
// ValidateAndPrepare implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) ValidateAndPrepare(blockAndPvtdata *ledger.BlockAndPvtData, doMVCCValidation bool) (
[]*txmgr.TxStatInfo, []byte, error,
) {
// Among ValidateAndPrepare(), PrepareExpiringKeys(), and
// RemoveStaleAndCommitPvtDataOfOldBlocks(), we can allow only one
// function to execute at a time. The reason is that each function calls
// LoadCommittedVersions() which would clear the existing entries in the
// transient buffer and load new entries (such a transient buffer is not
// applicable for the golevelDB). As a result, these three functions can
// interleave and nullify the optimization provided by the bulk read API.
// Once the ledger cache (FAB-103) is introduced and existing
// LoadCommittedVersions() is refactored to return a map, we can allow
// these three functions to execute parallely.
logger.Debugf("Waiting for purge mgr to finish the background job of computing expirying keys for the block")
txmgr.pvtdataPurgeMgr.WaitForPrepareToFinish()
txmgr.oldBlockCommit.Lock()
defer txmgr.oldBlockCommit.Unlock()
logger.Debug("lock acquired on oldBlockCommit for validating read set version against the committed version")
block := blockAndPvtdata.Block
logger.Debugf("Validating new block with num trans = [%d]", len(block.Data.Data))
batch, txstatsInfo, err := txmgr.validator.ValidateAndPrepareBatch(blockAndPvtdata, doMVCCValidation)
if err != nil {
txmgr.reset()
return nil, nil, err
}
txmgr.current = &current{block: block, batch: batch}
if err := txmgr.invokeNamespaceListeners(); err != nil {
txmgr.reset()
return nil, nil, err
}
updateBytesBuilder := &privacyenabledstate.UpdatesBytesBuilder{}
updateBytes, err := updateBytesBuilder.DeterministicBytesForPubAndHashUpdates(batch)
return txstatsInfo, updateBytes, err
}
// RemoveStaleAndCommitPvtDataOfOldBlocks implements method in interface `txmgmt.TxMgr`
// The following six operations are performed:
// (1) contructs the unique pvt data from the passed blocksPvtData
// (2) acquire a lock on oldBlockCommit
// (3) checks for stale pvtData by comparing [version, valueHash] and removes stale data
// (4) creates update batch from the the non-stale pvtData
// (5) update the BTL bookkeeping managed by the purge manager and update expiring keys.
// (6) commit the non-stale pvt data to the stateDB
// This function assumes that the passed input contains only transactions that had been
// marked "Valid". In the current design, kvledger (a single consumer of this function),
// filters out the data of "invalid" transactions and supplies the data for "valid" transactions only.
func (txmgr *LockBasedTxMgr) RemoveStaleAndCommitPvtDataOfOldBlocks(blocksPvtData map[uint64][]*ledger.TxPvtData) error {
// (0) Among ValidateAndPrepare(), PrepareExpiringKeys(), and
// RemoveStaleAndCommitPvtDataOfOldBlocks(), we can allow only one
// function to execute at a time. The reason is that each function calls
// LoadCommittedVersions() which would clear the existing entries in the
// transient buffer and load new entries (such a transient buffer is not
// applicable for the golevelDB). As a result, these three functions can
// interleave and nullify the optimization provided by the bulk read API.
// Once the ledger cache (FAB-103) is introduced and existing
// LoadCommittedVersions() is refactored to return a map, we can allow
// these three functions to execute parallely. However, we cannot remove
// the lock on oldBlockCommit as it is also used to avoid interleaving
// between Commit() and execution of this function for the correctness.
logger.Debug("Waiting for purge mgr to finish the background job of computing expirying keys for the block")
txmgr.pvtdataPurgeMgr.WaitForPrepareToFinish()
txmgr.oldBlockCommit.Lock()
defer txmgr.oldBlockCommit.Unlock()
logger.Debug("lock acquired on oldBlockCommit for committing pvtData of old blocks to state database")
// (1) as the blocksPvtData can contain multiple versions of pvtData for
// a given <ns, coll, key>, we need to find duplicate tuples with different
// versions and use the one with the higher version
logger.Debug("Constructing unique pvtData by removing duplicate entries")
uniquePvtData, err := constructUniquePvtData(blocksPvtData)
if len(uniquePvtData) == 0 || err != nil {
return err
}
// (3) remove the pvt data which does not matches the hashed
// value stored in the public state
logger.Debug("Finding and removing stale pvtData")
if err := uniquePvtData.findAndRemoveStalePvtData(txmgr.db); err != nil {
return err
}
// (4) create the update batch from the uniquePvtData
batch := uniquePvtData.transformToUpdateBatch()
// (5) update bookkeeping in the purge manager and update toPurgeList
// (i.e., the list of expiry keys). As the expiring keys would have
// been constructed during last PrepareExpiringKeys from commit, we need
// to update the list. This is because RemoveStaleAndCommitPvtDataOfOldBlocks
// may have added new data which might be eligible for expiry during the
// next regular block commit.
logger.Debug("Updating bookkeeping info in the purge manager")
if err := txmgr.pvtdataPurgeMgr.UpdateBookkeepingForPvtDataOfOldBlocks(batch.PvtUpdates); err != nil {
return err
}
// (6) commit the pvt data to the stateDB
logger.Debug("Committing updates to state database")
if err := txmgr.db.ApplyPrivacyAwareUpdates(batch, nil); err != nil {
return err
}
return nil
}
type uniquePvtDataMap map[privacyenabledstate.HashedCompositeKey]*privacyenabledstate.PvtKVWrite
func constructUniquePvtData(blocksPvtData map[uint64][]*ledger.TxPvtData) (uniquePvtDataMap, error) {
uniquePvtData := make(uniquePvtDataMap)
// go over the blocksPvtData to find duplicate <ns, coll, key>
// in the pvtWrites and use the one with the higher version number
for blkNum, blockPvtData := range blocksPvtData {
if err := uniquePvtData.updateUsingBlockPvtData(blockPvtData, blkNum); err != nil {
return nil, err
}
} // for each block
return uniquePvtData, nil
}
func (uniquePvtData uniquePvtDataMap) updateUsingBlockPvtData(blockPvtData []*ledger.TxPvtData, blkNum uint64) error {
for _, txPvtData := range blockPvtData {
ver := version.NewHeight(blkNum, txPvtData.SeqInBlock)
if err := uniquePvtData.updateUsingTxPvtData(txPvtData, ver); err != nil {
return err
}
} // for each tx
return nil
}
func (uniquePvtData uniquePvtDataMap) updateUsingTxPvtData(txPvtData *ledger.TxPvtData, ver *version.Height) error {
for _, nsPvtData := range txPvtData.WriteSet.NsPvtRwset {
if err := uniquePvtData.updateUsingNsPvtData(nsPvtData, ver); err != nil {
return err
}
} // for each ns
return nil
}
func (uniquePvtData uniquePvtDataMap) updateUsingNsPvtData(nsPvtData *rwset.NsPvtReadWriteSet, ver *version.Height) error {
for _, collPvtData := range nsPvtData.CollectionPvtRwset {
if err := uniquePvtData.updateUsingCollPvtData(collPvtData, nsPvtData.Namespace, ver); err != nil {
return err
}
} // for each coll
return nil
}
func (uniquePvtData uniquePvtDataMap) updateUsingCollPvtData(collPvtData *rwset.CollectionPvtReadWriteSet,
ns string, ver *version.Height) error {
kvRWSet := &kvrwset.KVRWSet{}
if err := proto.Unmarshal(collPvtData.Rwset, kvRWSet); err != nil {
return err
}
hashedCompositeKey := privacyenabledstate.HashedCompositeKey{
Namespace: ns,
CollectionName: collPvtData.CollectionName,
}
for _, kvWrite := range kvRWSet.Writes { // for each kv pair
hashedCompositeKey.KeyHash = string(util.ComputeStringHash(kvWrite.Key))
uniquePvtData.updateUsingPvtWrite(kvWrite, hashedCompositeKey, ver)
} // for each kv pair
return nil
}
func (uniquePvtData uniquePvtDataMap) updateUsingPvtWrite(pvtWrite *kvrwset.KVWrite,
hashedCompositeKey privacyenabledstate.HashedCompositeKey, ver *version.Height) {
pvtData, ok := uniquePvtData[hashedCompositeKey]
if !ok || pvtData.Version.Compare(ver) < 0 {
uniquePvtData[hashedCompositeKey] =
&privacyenabledstate.PvtKVWrite{
Key: pvtWrite.Key,
IsDelete: pvtWrite.IsDelete,
Value: pvtWrite.Value,
Version: ver,
}
}
}
func (uniquePvtData uniquePvtDataMap) findAndRemoveStalePvtData(db privacyenabledstate.DB) error {
// (1) load all committed versions
if err := uniquePvtData.loadCommittedVersionIntoCache(db); err != nil {
return err
}
// (2) find and remove the stale data
for hashedCompositeKey, pvtWrite := range uniquePvtData {
isStale, err := checkIfPvtWriteIsStale(&hashedCompositeKey, pvtWrite, db)
if err != nil {
return err
}
if isStale {
delete(uniquePvtData, hashedCompositeKey)
}
}
return nil
}
func (uniquePvtData uniquePvtDataMap) loadCommittedVersionIntoCache(db privacyenabledstate.DB) error {
// Note that ClearCachedVersions would not be called till we validate and commit these
// pvt data of old blocks. This is because only during the exclusive lock duration, we
// clear the cache and we have already acquired one before reaching here.
var hashedCompositeKeys []*privacyenabledstate.HashedCompositeKey
for hashedCompositeKey := range uniquePvtData {
// tempKey ensures a different pointer is added to the slice for each key
tempKey := hashedCompositeKey
hashedCompositeKeys = append(hashedCompositeKeys, &tempKey)
}
err := db.LoadCommittedVersionsOfPubAndHashedKeys(nil, hashedCompositeKeys)
if err != nil {
return err
}
return nil
}
func checkIfPvtWriteIsStale(hashedKey *privacyenabledstate.HashedCompositeKey,
kvWrite *privacyenabledstate.PvtKVWrite, db privacyenabledstate.DB) (bool, error) {
ns := hashedKey.Namespace
coll := hashedKey.CollectionName
keyHashBytes := []byte(hashedKey.KeyHash)
committedVersion, err := db.GetKeyHashVersion(ns, coll, keyHashBytes)
if err != nil {
return true, err
}
// for a deleted hashedKey, we would get a nil committed version. Note that
// the hashedKey was deleted because either it got expired or was deleted by
// the chaincode itself.
if committedVersion == nil {
return !kvWrite.IsDelete, nil
}
/*
TODO: FAB-12922
In the first round, we need to the check version of passed pvtData
against the version of pvtdata stored in the stateDB. In second round,
for the remaining pvtData, we need to check for stalenss using hashed
version. In the third round, for the still remaining pvtdata, we need
to check against hashed values. In each phase we would require to
perform bulkload of relevant data from the stateDB.
committedPvtData, err := db.GetPrivateData(ns, coll, kvWrite.Key)
if err != nil {
return false, err
}
if committedPvtData.Version.Compare(kvWrite.Version) > 0 {
return false, nil
}
*/
if version.AreSame(committedVersion, kvWrite.Version) {
return false, nil
}
// due to metadata updates, we could get a version
// mismatch between pvt kv write and the committed
// hashedKey. In this case, we must compare the hash
// of the value. If the hash matches, we should update
// the version number in the pvt kv write and return
// true as the validation result
vv, err := db.GetValueHash(ns, coll, keyHashBytes)
if err != nil {
return true, err
}
if bytes.Equal(vv.Value, util.ComputeHash(kvWrite.Value)) {
// if hash of value matches, update version
// and return true
kvWrite.Version = vv.Version // side effect
// (checkIfPvtWriteIsStale should not be updating the state)
return false, nil
}
return true, nil
}
func (uniquePvtData uniquePvtDataMap) transformToUpdateBatch() *privacyenabledstate.UpdateBatch {
batch := privacyenabledstate.NewUpdateBatch()
for hashedCompositeKey, pvtWrite := range uniquePvtData {
ns := hashedCompositeKey.Namespace
coll := hashedCompositeKey.CollectionName
if pvtWrite.IsDelete {
batch.PvtUpdates.Delete(ns, coll, pvtWrite.Key, pvtWrite.Version)
} else {
batch.PvtUpdates.Put(ns, coll, pvtWrite.Key, pvtWrite.Value, pvtWrite.Version)
}
}
return batch
}
func (txmgr *LockBasedTxMgr) invokeNamespaceListeners() error {
for _, listener := range txmgr.stateListeners {
stateUpdatesForListener := extractStateUpdates(txmgr.current.batch, listener.InterestedInNamespaces())
if len(stateUpdatesForListener) == 0 {
continue
}
txmgr.current.listeners = append(txmgr.current.listeners, listener)
committedStateQueryExecuter := &queryutil.QECombiner{
QueryExecuters: []queryutil.QueryExecuter{txmgr.db}}
postCommitQueryExecuter := &queryutil.QECombiner{
QueryExecuters: []queryutil.QueryExecuter{
&queryutil.UpdateBatchBackedQueryExecuter{UpdateBatch: txmgr.current.batch.PubUpdates.UpdateBatch},
txmgr.db,
},
}
trigger := &ledger.StateUpdateTrigger{
LedgerID: txmgr.ledgerid,
StateUpdates: stateUpdatesForListener,
CommittingBlockNum: txmgr.current.blockNum(),
CommittedStateQueryExecutor: committedStateQueryExecuter,
PostCommitQueryExecutor: postCommitQueryExecuter,
}
if err := listener.HandleStateUpdates(trigger); err != nil {
return err
}
logger.Debugf("Invoking listener for state changes:%s", listener)
}
return nil
}
// Shutdown implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) Shutdown() {
// wait for background go routine to finish else the timing issue causes a nil pointer inside goleveldb code
// see FAB-11974
txmgr.pvtdataPurgeMgr.WaitForPrepareToFinish()
txmgr.db.Close()
}
// Commit implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) Commit() error {
// we need to acquire a lock on oldBlockCommit. The following are the two reasons:
// (1) the DeleteExpiredAndUpdateBookkeeping() would perform incorrect operation if
// toPurgeList is updated by RemoveStaleAndCommitPvtDataOfOldBlocks().
// (2) RemoveStaleAndCommitPvtDataOfOldBlocks computes the update
// batch based on the current state and if we allow regular block commits at the
// same time, the former may overwrite the newer versions of the data and we may
// end up with an incorrect update batch.
txmgr.oldBlockCommit.Lock()
defer txmgr.oldBlockCommit.Unlock()
logger.Debug("lock acquired on oldBlockCommit for committing regular updates to state database")
// When using the purge manager for the first block commit after peer start, the asynchronous function
// 'PrepareForExpiringKeys' is invoked in-line. However, for the subsequent blocks commits, this function is invoked
// in advance for the next block
if !txmgr.pvtdataPurgeMgr.usedOnce {
txmgr.pvtdataPurgeMgr.PrepareForExpiringKeys(txmgr.current.blockNum())
txmgr.pvtdataPurgeMgr.usedOnce = true
}
defer func() {
txmgr.pvtdataPurgeMgr.PrepareForExpiringKeys(txmgr.current.blockNum() + 1)
logger.Debugf("launched the background routine for preparing keys to purge with the next block")
txmgr.reset()
}()
logger.Debugf("Committing updates to state database")
if txmgr.current == nil {
panic("validateAndPrepare() method should have been called before calling commit()")
}
if err := txmgr.pvtdataPurgeMgr.DeleteExpiredAndUpdateBookkeeping(
txmgr.current.batch.PvtUpdates, txmgr.current.batch.HashUpdates); err != nil {
return err
}
commitHeight := version.NewHeight(txmgr.current.blockNum(), txmgr.current.maxTxNumber())
txmgr.commitRWLock.Lock()
logger.Debugf("Write lock acquired for committing updates to state database")
if err := txmgr.db.ApplyPrivacyAwareUpdates(txmgr.current.batch, commitHeight); err != nil {
txmgr.commitRWLock.Unlock()
return err
}
txmgr.commitRWLock.Unlock()
// only while holding a lock on oldBlockCommit, we should clear the cache as the
// cache is being used by the old pvtData committer to load the version of
// hashedKeys. Also, note that the PrepareForExpiringKeys uses the cache.
txmgr.clearCache()
logger.Debugf("Updates committed to state database and the write lock is released")
// purge manager should be called (in this call the purge mgr removes the expiry entries from schedules) after committing to statedb
if err := txmgr.pvtdataPurgeMgr.BlockCommitDone(); err != nil {
return err
}
// In the case of error state listeners will not recieve this call - instead a peer panic is caused by the ledger upon receiveing
// an error from this function
txmgr.updateStateListeners()
return nil
}
// Rollback implements method in interface `txmgmt.TxMgr`
func (txmgr *LockBasedTxMgr) Rollback() {
txmgr.reset()
}
// clearCache empty the cache maintained by the statedb implementation
func (txmgr *LockBasedTxMgr) clearCache() {
if txmgr.db.IsBulkOptimizable() {
txmgr.db.ClearCachedVersions()
}
}
// ShouldRecover implements method in interface kvledger.Recoverer
func (txmgr *LockBasedTxMgr) ShouldRecover(lastAvailableBlock uint64) (bool, uint64, error) {
savepoint, err := txmgr.GetLastSavepoint()
if err != nil {
return false, 0, err
}
if savepoint == nil {
return true, 0, nil
}
return savepoint.BlockNum != lastAvailableBlock, savepoint.BlockNum + 1, nil
}
// Name returns the name of the database that manages all active states.
func (txmgr *LockBasedTxMgr) Name() string {
return "state"
}
// CommitLostBlock implements method in interface kvledger.Recoverer
func (txmgr *LockBasedTxMgr) CommitLostBlock(blockAndPvtdata *ledger.BlockAndPvtData) error {
block := blockAndPvtdata.Block
logger.Debugf("Constructing updateSet for the block %d", block.Header.Number)
if _, _, err := txmgr.ValidateAndPrepare(blockAndPvtdata, false); err != nil {
return err
}
// log every 1000th block at Info level so that statedb rebuild progress can be tracked in production envs.
if block.Header.Number%1000 == 0 {
logger.Infof("Recommitting block [%d] to state database", block.Header.Number)
} else {
logger.Debugf("Recommitting block [%d] to state database", block.Header.Number)
}
return txmgr.Commit()
}
func extractStateUpdates(batch *privacyenabledstate.UpdateBatch, namespaces []string) ledger.StateUpdates {
stateupdates := make(ledger.StateUpdates)
for _, namespace := range namespaces {
updatesMap := batch.PubUpdates.GetUpdates(namespace)
var kvwrites []*kvrwset.KVWrite
for key, versionedValue := range updatesMap {
kvwrites = append(kvwrites, &kvrwset.KVWrite{Key: key, IsDelete: versionedValue.Value == nil, Value: versionedValue.Value})
if len(kvwrites) > 0 {
stateupdates[namespace] = kvwrites
}
}
}
return stateupdates
}
func (txmgr *LockBasedTxMgr) updateStateListeners() {
for _, l := range txmgr.current.listeners {
l.StateCommitDone(txmgr.ledgerid)
}
}
func (txmgr *LockBasedTxMgr) reset() {
txmgr.current = nil
}
// pvtdataPurgeMgr wraps the actual purge manager and an additional flag 'usedOnce'
// for usage of this additional flag, see the relevant comments in the txmgr.Commit() function above
type pvtdataPurgeMgr struct {
pvtstatepurgemgmt.PurgeMgr
usedOnce bool
}
Go
1
https://gitee.com/hyperledger-fabric-gm/fabric.git
git@gitee.com:hyperledger-fabric-gm/fabric.git
hyperledger-fabric-gm
fabric
fabric
v1.4.9

搜索帮助