1 Star 0 Fork 0

妥協/fabric

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
scc.go 18.62 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567
/*
Copyright IBM Corp. All Rights Reserved.
SPDX-License-Identifier: Apache-2.0
*/
package lifecycle
import (
"fmt"
"regexp"
"github.com/hyperledger/fabric/common/chaincode"
"github.com/hyperledger/fabric/common/channelconfig"
"github.com/hyperledger/fabric/core/aclmgmt"
"github.com/hyperledger/fabric/core/chaincode/persistence"
persistenceintf "github.com/hyperledger/fabric/core/chaincode/persistence/intf"
"github.com/hyperledger/fabric/core/chaincode/shim"
"github.com/hyperledger/fabric/core/dispatcher"
cb "github.com/hyperledger/fabric/protos/common"
pb "github.com/hyperledger/fabric/protos/peer"
lb "github.com/hyperledger/fabric/protos/peer/lifecycle"
"github.com/golang/protobuf/proto"
"github.com/pkg/errors"
"go.uber.org/zap/zapcore"
)
const (
// LifecycleNamespace is the namespace in the statedb where lifecycle
// information is stored
LifecycleNamespace = "_lifecycle"
// InstallChaincodeFuncName is the chaincode function name used to install
// a chaincode
InstallChaincodeFuncName = "InstallChaincode"
// QueryInstalledChaincodeFuncName is the chaincode function name used to
// query an installed chaincode
QueryInstalledChaincodeFuncName = "QueryInstalledChaincode"
// QueryInstalledChaincodesFuncName is the chaincode function name used to
// query all installed chaincodes
QueryInstalledChaincodesFuncName = "QueryInstalledChaincodes"
// ApproveChaincodeDefinitionForMyOrgFuncName is the chaincode function name
// used to approve a chaincode definition for execution by the user's own org
ApproveChaincodeDefinitionForMyOrgFuncName = "ApproveChaincodeDefinitionForMyOrg"
// QueryApprovalStatusFuncName is the chaincode function name used to query
// the approval status for a given definition over a given set of orgs
QueryApprovalStatusFuncName = "QueryApprovalStatus"
// CommitChaincodeDefinitionFuncName is the chaincode function name used to
// 'commit' (previously 'instantiate') a chaincode in a channel.
CommitChaincodeDefinitionFuncName = "CommitChaincodeDefinition"
// QueryChaincodeDefinitionFuncName is the chaincode function name used to
// query the committed chaincode definitions in a channel.
QueryChaincodeDefinitionFuncName = "QueryChaincodeDefinition"
// QueryNamespaceDefinitionsFuncName is the chaincode function name used
// to query which namespaces are currently defined and what type those
// namespaces are.
QueryNamespaceDefinitionsFuncName = "QueryNamespaceDefinitions"
)
// SCCFunctions provides a backing implementation with concrete arguments
// for each of the SCC functions
type SCCFunctions interface {
// InstallChaincode persists a chaincode definition to disk
InstallChaincode([]byte) (*chaincode.InstalledChaincode, error)
// QueryInstalledChaincode returns the hash for a given name and version of an installed chaincode
QueryInstalledChaincode(packageID persistenceintf.PackageID) (*chaincode.InstalledChaincode, error)
// QueryInstalledChaincodes returns the currently installed chaincodes
QueryInstalledChaincodes() (chaincodes []chaincode.InstalledChaincode, err error)
// ApproveChaincodeDefinitionForOrg records a chaincode definition into this org's implicit collection.
ApproveChaincodeDefinitionForOrg(chname, ccname string, cd *ChaincodeDefinition, packageID persistenceintf.PackageID, publicState ReadableState, orgState ReadWritableState) error
// QueryApprovalStatus returns an array of boolean to signal whether the orgs
// whose orgStates was supplied as argument have approveed the specified definition
QueryApprovalStatus(chname, ccname string, cd *ChaincodeDefinition, publicState ReadWritableState, orgStates []OpaqueState) ([]bool, error)
// CommitChaincodeDefinition records a new chaincode definition into the public state and returns the orgs which agreed with that definition.
CommitChaincodeDefinition(chname, ccname string, cd *ChaincodeDefinition, publicState ReadWritableState, orgStates []OpaqueState) ([]bool, error)
// QueryChaincodeDefinition reads a chaincode definition from the public state.
QueryChaincodeDefinition(name string, publicState ReadableState) (*ChaincodeDefinition, error)
// QueryNamespaceDefinitions returns all defined namespaces
QueryNamespaceDefinitions(publicState RangeableState) (map[string]string, error)
}
//go:generate counterfeiter -o mock/channel_config_source.go --fake-name ChannelConfigSource . ChannelConfigSource
// ChannelConfigSource provides a way to retrieve the channel config for a given
// channel ID.
type ChannelConfigSource interface {
// GetStableChannelConfig returns the channel config for a given channel id.
// Note, it is a stable bundle, which means it will not be updated, even if
// the channel is, so it should be discarded after use.
GetStableChannelConfig(channelID string) channelconfig.Resources
}
// SCC implements the required methods to satisfy the chaincode interface.
// It routes the invocation calls to the backing implementations.
type SCC struct {
OrgMSPID string
ACLProvider aclmgmt.ACLProvider
ChannelConfigSource ChannelConfigSource
// Functions provides the backing implementation of lifecycle.
Functions SCCFunctions
// Dispatcher handles the rote protobuf boilerplate for unmarshaling/marshaling
// the inputs and outputs of the SCC functions.
Dispatcher *dispatcher.Dispatcher
}
// Name returns "_lifecycle"
func (scc *SCC) Name() string {
return LifecycleNamespace
}
// Path returns "github.com/hyperledger/fabric/core/chaincode/lifecycle"
func (scc *SCC) Path() string {
return "github.com/hyperledger/fabric/core/chaincode/lifecycle"
}
// InitArgs returns nil
func (scc *SCC) InitArgs() [][]byte {
return nil
}
// Chaincode returns a reference to itself
func (scc *SCC) Chaincode() shim.Chaincode {
return scc
}
// InvokableExternal returns true
func (scc *SCC) InvokableExternal() bool {
return true
}
// InvokableCC2CC returns true
func (scc *SCC) InvokableCC2CC() bool {
return true
}
// Enabled returns true
func (scc *SCC) Enabled() bool {
return true
}
// Init is mostly useless for system chaincodes and always returns success
func (scc *SCC) Init(stub shim.ChaincodeStubInterface) pb.Response {
return shim.Success(nil)
}
// Invoke takes chaincode invocation arguments and routes them to the correct
// underlying lifecycle operation. All functions take a single argument of
// type marshaled lb.<FunctionName>Args and return a marshaled lb.<FunctionName>Result
func (scc *SCC) Invoke(stub shim.ChaincodeStubInterface) pb.Response {
args := stub.GetArgs()
if len(args) == 0 {
return shim.Error("lifecycle scc must be invoked with arguments")
}
if len(args) != 2 {
return shim.Error(fmt.Sprintf("lifecycle scc operations require exactly two arguments but received %d", len(args)))
}
var ac channelconfig.Application
if channelID := stub.GetChannelID(); channelID != "" {
channelConfig := scc.ChannelConfigSource.GetStableChannelConfig(channelID)
if channelConfig == nil {
return shim.Error(fmt.Sprintf("could not get channelconfig for channel '%s'", channelID))
}
var ok bool
ac, ok = channelConfig.ApplicationConfig()
if !ok {
return shim.Error(fmt.Sprintf("could not get application config for channel '%s'", channelID))
}
if !ac.Capabilities().LifecycleV20() {
return shim.Error(fmt.Sprintf("cannot use new lifecycle for channel '%s' as it does not have the required capabilities enabled", channelID))
}
}
// Handle ACL:
sp, err := stub.GetSignedProposal()
if err != nil {
return shim.Error(fmt.Sprintf("Failed getting signed proposal from stub: [%s]", err))
}
err = scc.ACLProvider.CheckACL(fmt.Sprintf("%s/%s", LifecycleNamespace, args[0]), stub.GetChannelID(), sp)
if err != nil {
return shim.Error(fmt.Sprintf("Failed to authorize invocation due to failed ACL check: %s", err))
}
outputBytes, err := scc.Dispatcher.Dispatch(
args[1],
string(args[0]),
&Invocation{
ApplicationConfig: ac,
SCC: scc,
Stub: stub,
},
)
if err != nil {
switch err.(type) {
case ErrNamespaceNotDefined, persistence.CodePackageNotFoundErr:
return pb.Response{
Status: 404,
Message: err.Error(),
}
default:
return shim.Error(fmt.Sprintf("failed to invoke backing implementation of '%s': %s", string(args[0]), err.Error()))
}
}
return shim.Success(outputBytes)
}
type Invocation struct {
ApplicationConfig channelconfig.Application // Note this may be nil
Stub shim.ChaincodeStubInterface
SCC *SCC
}
// InstallChaincode is a SCC function that may be dispatched to which routes
// to the underlying lifecycle implementation.
func (i *Invocation) InstallChaincode(input *lb.InstallChaincodeArgs) (proto.Message, error) {
if logger.IsEnabledFor(zapcore.DebugLevel) {
end := 35
if len(input.ChaincodeInstallPackage) < end {
end = len(input.ChaincodeInstallPackage)
}
// the first tens of bytes contain the (compressed) portion
// of the package metadata and so they'll be different across
// different packages, acting as a package fingerprint useful
// to identify various packages from the content
packageFingerprint := input.ChaincodeInstallPackage[0:end]
logger.Debugf("received invocation of InstallChaincode for install package %x...",
packageFingerprint,
)
}
installedCC, err := i.SCC.Functions.InstallChaincode(input.ChaincodeInstallPackage)
if err != nil {
return nil, err
}
return &lb.InstallChaincodeResult{
Label: installedCC.Label,
PackageId: installedCC.PackageID.String(),
}, nil
}
// QueryInstalledChaincode is a SCC function that may be dispatched to which
// routes to the underlying lifecycle implementation.
func (i *Invocation) QueryInstalledChaincode(input *lb.QueryInstalledChaincodeArgs) (proto.Message, error) {
logger.Debugf("received invocation of QueryInstalledChaincode for install package ID '%s'",
input.PackageId,
)
chaincode, err := i.SCC.Functions.QueryInstalledChaincode(persistenceintf.PackageID(input.PackageId))
if err != nil {
return nil, err
}
return &lb.QueryInstalledChaincodeResult{
Label: chaincode.Label,
PackageId: chaincode.PackageID.String(),
}, nil
}
// QueryInstalledChaincodes is a SCC function that may be dispatched to which
// routes to the underlying lifecycle implementation.
func (i *Invocation) QueryInstalledChaincodes(input *lb.QueryInstalledChaincodesArgs) (proto.Message, error) {
logger.Debugf("received invocation of QueryInstalledChaincodes")
chaincodes, err := i.SCC.Functions.QueryInstalledChaincodes()
if err != nil {
return nil, err
}
result := &lb.QueryInstalledChaincodesResult{}
for _, chaincode := range chaincodes {
result.InstalledChaincodes = append(
result.InstalledChaincodes,
&lb.QueryInstalledChaincodesResult_InstalledChaincode{
Label: chaincode.Label,
PackageId: chaincode.PackageID.String(),
})
}
return result, nil
}
// ApproveChaincodeDefinitionForMyOrg is a SCC function that may be dispatched
// to which routes to the underlying lifecycle implementation.
func (i *Invocation) ApproveChaincodeDefinitionForMyOrg(input *lb.ApproveChaincodeDefinitionForMyOrgArgs) (proto.Message, error) {
if err := validateNameVersion(input.Name, input.Version); err != nil {
return nil, err
}
collectionName := ImplicitCollectionNameForOrg(i.SCC.OrgMSPID)
var collectionConfig []*cb.CollectionConfig
if input.Collections != nil {
collectionConfig = input.Collections.Config
}
var packageID persistenceintf.PackageID
if input.Source != nil {
switch source := input.Source.Type.(type) {
case *lb.ChaincodeSource_LocalPackage:
packageID = persistenceintf.PackageID(source.LocalPackage.PackageId)
case *lb.ChaincodeSource_Unavailable_:
default:
}
}
cd := &ChaincodeDefinition{
Sequence: input.Sequence,
EndorsementInfo: &lb.ChaincodeEndorsementInfo{
Version: input.Version,
EndorsementPlugin: input.EndorsementPlugin,
InitRequired: input.InitRequired,
},
ValidationInfo: &lb.ChaincodeValidationInfo{
ValidationPlugin: input.ValidationPlugin,
ValidationParameter: input.ValidationParameter,
},
Collections: &cb.CollectionConfigPackage{
Config: collectionConfig,
},
}
logger.Debugf("received invocation of ApproveChaincodeDefinitionForMyOrg on channel '%s' for definition '%s'",
i.Stub.GetChannelID(),
cd,
)
if err := i.SCC.Functions.ApproveChaincodeDefinitionForOrg(
i.Stub.GetChannelID(),
input.Name,
cd,
packageID,
i.Stub,
&ChaincodePrivateLedgerShim{
Collection: collectionName,
Stub: i.Stub,
},
); err != nil {
return nil, err
}
return &lb.ApproveChaincodeDefinitionForMyOrgResult{}, nil
}
// QueryApprovalStatus is a SCC function that may be dispatched to the underlying
// lifecycle implementation
func (i *Invocation) QueryApprovalStatus(input *lb.QueryApprovalStatusArgs) (proto.Message, error) {
if i.ApplicationConfig == nil {
return nil, errors.Errorf("no application config for channel '%s'", i.Stub.GetChannelID())
}
orgs := i.ApplicationConfig.Organizations()
opaqueStates := make([]OpaqueState, 0, len(orgs))
orgNames := make([]string, 0, len(orgs))
for _, org := range orgs {
orgNames = append(orgNames, org.MSPID())
opaqueStates = append(opaqueStates, &ChaincodePrivateLedgerShim{
Collection: ImplicitCollectionNameForOrg(org.MSPID()),
Stub: i.Stub,
})
}
cd := &ChaincodeDefinition{
Sequence: input.Sequence,
EndorsementInfo: &lb.ChaincodeEndorsementInfo{
Version: input.Version,
EndorsementPlugin: input.EndorsementPlugin,
InitRequired: input.InitRequired,
},
ValidationInfo: &lb.ChaincodeValidationInfo{
ValidationPlugin: input.ValidationPlugin,
ValidationParameter: input.ValidationParameter,
},
Collections: input.Collections,
}
logger.Debugf("received invocation of QueryApprovalStatus on channel '%s' for definition '%s'",
i.Stub.GetChannelID(),
cd,
)
approved, err := i.SCC.Functions.QueryApprovalStatus(
i.Stub.GetChannelID(),
input.Name,
cd,
i.Stub,
opaqueStates,
)
if err != nil {
return nil, err
}
orgApproval := make(map[string]bool)
for i, org := range orgNames {
orgApproval[org] = approved[i]
}
return &lb.QueryApprovalStatusResults{
Approved: orgApproval,
}, nil
}
// CommitChaincodeDefinition is a SCC function that may be dispatched
// to which routes to the underlying lifecycle implementation.
func (i *Invocation) CommitChaincodeDefinition(input *lb.CommitChaincodeDefinitionArgs) (proto.Message, error) {
if err := validateNameVersion(input.Name, input.Version); err != nil {
return nil, err
}
if i.ApplicationConfig == nil {
return nil, errors.Errorf("no application config for channel '%s'", i.Stub.GetChannelID())
}
orgs := i.ApplicationConfig.Organizations()
opaqueStates := make([]OpaqueState, 0, len(orgs))
myOrgIndex := -1
for _, org := range orgs {
opaqueStates = append(opaqueStates, &ChaincodePrivateLedgerShim{
Collection: ImplicitCollectionNameForOrg(org.MSPID()),
Stub: i.Stub,
})
if org.MSPID() == i.SCC.OrgMSPID {
myOrgIndex = len(opaqueStates) - 1
}
}
if myOrgIndex == -1 {
return nil, errors.Errorf("impossibly, this peer's org is processing requests for a channel it is not a member of")
}
cd := &ChaincodeDefinition{
Sequence: input.Sequence,
EndorsementInfo: &lb.ChaincodeEndorsementInfo{
Version: input.Version,
EndorsementPlugin: input.EndorsementPlugin,
InitRequired: input.InitRequired,
},
ValidationInfo: &lb.ChaincodeValidationInfo{
ValidationPlugin: input.ValidationPlugin,
ValidationParameter: input.ValidationParameter,
},
Collections: input.Collections,
}
logger.Debugf("received invocation of CommitChaincodeDefinition on channel '%s' for definition '%s'",
i.Stub.GetChannelID(),
cd,
)
agreement, err := i.SCC.Functions.CommitChaincodeDefinition(
i.Stub.GetChannelID(),
input.Name,
cd,
i.Stub,
opaqueStates,
)
if err != nil {
return nil, err
}
if !agreement[myOrgIndex] {
return nil, errors.Errorf("chaincode definition not agreed to by this org (%s)", i.SCC.OrgMSPID)
}
return &lb.CommitChaincodeDefinitionResult{}, nil
}
// QueryChaincodeDefinition is a SCC function that may be dispatched
// to which routes to the underlying lifecycle implementation.
func (i *Invocation) QueryChaincodeDefinition(input *lb.QueryChaincodeDefinitionArgs) (proto.Message, error) {
logger.Debugf("received invocation of QueryChaincodeDefinition on channel '%s' for chaincode '%s'",
i.Stub.GetChannelID(),
input.Name,
)
definedChaincode, err := i.SCC.Functions.QueryChaincodeDefinition(input.Name, i.Stub)
if err != nil {
return nil, err
}
return &lb.QueryChaincodeDefinitionResult{
Sequence: definedChaincode.Sequence,
Version: definedChaincode.EndorsementInfo.Version,
EndorsementPlugin: definedChaincode.EndorsementInfo.EndorsementPlugin,
ValidationPlugin: definedChaincode.ValidationInfo.ValidationPlugin,
ValidationParameter: definedChaincode.ValidationInfo.ValidationParameter,
InitRequired: definedChaincode.EndorsementInfo.InitRequired,
Collections: definedChaincode.Collections,
}, nil
}
// QueryNamespaceDefinitions is a SCC function that may be dispatched
// to which routes to the underlying lifecycle implementation.
func (i *Invocation) QueryNamespaceDefinitions(input *lb.QueryNamespaceDefinitionsArgs) (proto.Message, error) {
logger.Debugf("received invocation of QueryNamespaceDefinitions on channel '%s'",
i.Stub.GetChannelID(),
)
namespaces, err := i.SCC.Functions.QueryNamespaceDefinitions(&ChaincodePublicLedgerShim{ChaincodeStubInterface: i.Stub})
if err != nil {
return nil, err
}
result := map[string]*lb.QueryNamespaceDefinitionsResult_Namespace{}
for namespace, nType := range namespaces {
result[namespace] = &lb.QueryNamespaceDefinitionsResult_Namespace{
Type: nType,
}
}
return &lb.QueryNamespaceDefinitionsResult{
Namespaces: result,
}, nil
}
var (
// NOTE these regular expressions should stay in sync with those defined in
// core/scc/lscc/lscc.go until LSCC has been removed.
chaincodeNameRegExp = regexp.MustCompile("^[a-zA-Z0-9]+([-_][a-zA-Z0-9]+)*$")
chaincodeVersionRegExp = regexp.MustCompile("^[A-Za-z0-9_.+-]+$")
)
func validateNameVersion(name, version string) error {
if !chaincodeNameRegExp.MatchString(name) {
return errors.Errorf("invalid chaincode name '%s'. Names can only consist of alphanumerics, '_', and '-'", name)
}
if !chaincodeVersionRegExp.MatchString(version) {
return errors.Errorf("invalid chaincode version '%s'. Versions can only consist of alphanumerics, '_', '-', '+', and '.'", version)
}
systemChaincodeNames := map[string]struct{}{
"cscc": {},
"escc": {},
"lscc": {},
"qscc": {},
"vscc": {},
}
if _, ok := systemChaincodeNames[name]; ok {
return errors.Errorf("chaincode name '%s' is the name of a system chaincode", name)
}
return nil
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
1
https://gitee.com/liurenhao/fabric.git
git@gitee.com:liurenhao/fabric.git
liurenhao
fabric
fabric
v2.0.0-alpha

搜索帮助