63 Star 181 Fork 3

Gitee 极速下载 / hyperledger-fabric

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
此仓库是为了提升国内下载速度的镜像仓库,每日同步一次。 原始仓库: https://github.com/hyperledger/fabric
克隆/下载
lockbased_txmgr.go 20.91 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551
/*
Copyright IBM Corp. All Rights Reserved.
SPDX-License-Identifier: Apache-2.0
*/
package lockbasedtxmgr
import (
"bytes"
"sync"
"github.com/golang/protobuf/proto"
"github.com/hyperledger/fabric/common/flogging"
"github.com/hyperledger/fabric/core/ledger"
"github.com/hyperledger/fabric/core/ledger/kvledger/bookkeeping"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/privacyenabledstate"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/pvtstatepurgemgmt"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/queryutil"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/txmgr"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/validator"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/validator/valimpl"
"github.com/hyperledger/fabric/core/ledger/kvledger/txmgmt/version"
"github.com/hyperledger/fabric/core/ledger/pvtdatapolicy"
"github.com/hyperledger/fabric/core/ledger/util"
"github.com/hyperledger/fabric/protos/common"
"github.com/hyperledger/fabric/protos/ledger/rwset"
"github.com/hyperledger/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
lastCommittedBlockNum uint64
}
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, error,
) {
block := blockAndPvtdata.Block
logger.Debugf("Waiting for purge mgr to finish the background job of computing expirying keys for the block")
txmgr.pvtdataPurgeMgr.WaitForPrepareToFinish()
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, err
}
txmgr.current = &current{block: block, batch: batch}
if err := txmgr.invokeNamespaceListeners(); err != nil {
txmgr.reset()
return nil, err
}
return txstatsInfo, nil
}
// 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 prepare 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, this assumption holds true as we store
// missing data info about only valid transactions. Further, gossip supplies only the
// missing pvtData of valid transactions. If these two assumptions are broken due to some bug,
// we are still safe from data consistency point of view as we match the version and the
// value hashes stored in the stateDB before committing the value. However, if pvtData of
// a tuple <ns, Coll, key> is passed for two (or more) transactions with one as valid and
// another as invalid transaction, we might miss to store a missing data forever if the
// version# of invalid tx is greater than the valid tx (as per our logic employed in
// constructUniquePvtData(). Other than a bug, there is another scenario in which this
// function might receive pvtData of both valid and invalid tx. Such a scenario is explained
// in FAB-12924 and is related to state fork and rebuilding ledger state.
func (txmgr *LockBasedTxMgr) RemoveStaleAndCommitPvtDataOfOldBlocks(blocksPvtData map[uint64][]*ledger.TxPvtData) error {
// (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
uniquePvtData, err := constructUniquePvtData(blocksPvtData)
if len(uniquePvtData) == 0 || err != nil {
return err
}
// (2) acquire a lock on oldBlockCommit. If the regular block commit has already
// acquired this lock, commit of old blocks' pvtData cannot proceed until the lock
// is released. This is required as the PrepareForExpiringKeys() used in step (5)
// of this function might affect the result of DeleteExpiredAndUpdateBookkeeping()
// in Commit()
txmgr.oldBlockCommit.Lock()
defer txmgr.oldBlockCommit.Unlock()
// (3) remove the pvt data which does not matches the hashed
// value stored in the public state
if err := uniquePvtData.findAndRemoveStalePvtData(txmgr.db); err != nil {
return err
}
// (4) create the update batch from the uniquePvtData
batch := uniquePvtData.transformToUpdateBatch()
// (5) update booking in the purge manager and prepare expiring keys.
// Though the expiring keys would have been loaded in memory during last
// PrepareExpiringKeys from Commit but we rerun this here because,
// RemoveStaleAndCommitPvtDataOfOldBlocks may have added new data which might be
// eligible for expiry during the next regular block commit.
if err := txmgr.pvtdataPurgeMgr.UpdateBookkeepingForPvtDataOfOldBlocks(batch.PvtUpdates); err != nil {
return err
}
if txmgr.lastCommittedBlockNum > 0 {
txmgr.pvtdataPurgeMgr.PrepareForExpiringKeys(txmgr.lastCommittedBlockNum + 1)
}
// (6) commit the pvt data to the stateDB
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 {
// the regular block validation might have populate the cache already. In that scenario,
// this call would be adding more entries to the existing cache. However, it does not affect
// the correctness of regular block validation. If an entry already exist for a given
// hashedCompositeKey, cache would not be updated.
// 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 {
hashedCompositeKeys = append(hashedCompositeKeys, &hashedCompositeKey)
}
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 && kvWrite.IsDelete {
return false, 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. This is required because
// the DeleteExpiredAndUpdateBookkeeping() would perform incorrect operation if
// PrepareForExpiringKeys() in RemoveStaleAndCommitPvtDataOfOldBlocks() is allowed to
// execute parallely. 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()
// 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()
txmgr.lastCommittedBlockNum = txmgr.current.blockNum()
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
}
// 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
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/mirrors/hyperledger-fabric.git
git@gitee.com:mirrors/hyperledger-fabric.git
mirrors
hyperledger-fabric
hyperledger-fabric
v1.4.0-rc1

搜索帮助

344bd9b3 5694891 D2dac590 5694891