Skip to content

Commit 8417c0e

Browse files
C0rWinyacovm
authored andcommitted
[FAB-1288]: Expose gossip API for cscc.
Add gossip service entity to encapsulate gossip + state instance. Make service instance a singelton and add JoinChannel method so cscc will be able to leverage it. Change-Id: I38233781276d538f861e472a427a1df12887c887 Signed-off-by: Artem Barger <[email protected]> Signed-off-by: Yacov Manevich <[email protected]>
1 parent ce296d2 commit 8417c0e

File tree

5 files changed

+241
-49
lines changed

5 files changed

+241
-49
lines changed

core/committer/noopssinglechain/client.go

+22-43
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,13 @@ limitations under the License.
1717
package noopssinglechain
1818

1919
import (
20+
"fmt"
2021
"time"
2122

2223
"github.com/golang/protobuf/proto"
2324
"github.com/hyperledger/fabric/core/chaincode"
2425
"github.com/hyperledger/fabric/core/committer"
2526
"github.com/hyperledger/fabric/core/ledger/kvledger"
26-
"github.com/hyperledger/fabric/core/util"
2727
"github.com/hyperledger/fabric/events/producer"
2828
"github.com/hyperledger/fabric/protos/common"
2929
"github.com/hyperledger/fabric/protos/orderer"
@@ -33,14 +33,10 @@ import (
3333
"golang.org/x/net/context"
3434
"google.golang.org/grpc"
3535

36-
"fmt"
37-
3836
"github.com/hyperledger/fabric/core/ledger"
3937
"github.com/hyperledger/fabric/core/peer"
40-
"github.com/hyperledger/fabric/gossip/gossip"
41-
"github.com/hyperledger/fabric/gossip/integration"
4238
gossip_proto "github.com/hyperledger/fabric/gossip/proto"
43-
"github.com/hyperledger/fabric/gossip/state"
39+
"github.com/hyperledger/fabric/gossip/service"
4440
pb "github.com/hyperledger/fabric/protos/peer"
4541
)
4642

@@ -57,11 +53,9 @@ type DeliverService struct {
5753
client orderer.AtomicBroadcast_DeliverClient
5854
windowSize uint64
5955
unAcknowledged uint64
60-
committer *committer.LedgerCommitter
6156

62-
stateProvider state.GossipStateProvider
63-
gossip gossip.Gossip
64-
conn *grpc.ClientConn
57+
chainID string
58+
conn *grpc.ClientConn
6559
}
6660

6761
// StopDeliveryService sends stop to the delivery service reference
@@ -73,25 +67,22 @@ func StopDeliveryService(service *DeliverService) {
7367

7468
// NewDeliverService construction function to create and initilize
7569
// delivery service instance
76-
func NewDeliverService(chainID string, address string, grpcServer *grpc.Server) *DeliverService {
70+
func NewDeliverService(chainID string) *DeliverService {
7771
if viper.GetBool("peer.committer.enabled") {
7872
logger.Infof("Creating committer for single noops endorser")
79-
8073
deliverService := &DeliverService{
8174
// Instance of RawLedger
82-
committer: committer.NewLedgerCommitter(kvledger.GetLedger(chainID)),
75+
chainID: chainID,
8376
windowSize: 10,
8477
}
8578

86-
deliverService.initStateProvider(address, grpcServer)
87-
8879
return deliverService
8980
}
9081
logger.Infof("Committer disabled")
9182
return nil
9283
}
9384

94-
func (d *DeliverService) startDeliver() error {
85+
func (d *DeliverService) startDeliver(committer committer.Committer) error {
9586
logger.Info("Starting deliver service client")
9687
err := d.initDeliver()
9788

@@ -100,7 +91,7 @@ func (d *DeliverService) startDeliver() error {
10091
return err
10192
}
10293

103-
height, err := d.committer.LedgerHeight()
94+
height, err := committer.LedgerHeight()
10495
if err != nil {
10596
logger.Errorf("Can't get legder height from committer [%s]", err)
10697
return err
@@ -153,36 +144,24 @@ func (d *DeliverService) stopDeliver() {
153144
}
154145
}
155146

156-
func (d *DeliverService) initStateProvider(address string, grpcServer *grpc.Server) error {
157-
bootstrap := viper.GetStringSlice("peer.gossip.bootstrap")
158-
logger.Debug("Initializing state provideer, endpoint = ", address, " bootstrap set = ", bootstrap)
159-
160-
gossip, gossipComm := integration.NewGossipComponent(address, grpcServer, bootstrap...)
161-
162-
d.gossip = gossip
163-
d.stateProvider = state.NewGossipStateProvider(gossip, gossipComm, d.committer)
164-
return nil
165-
}
166-
167-
// Start the delivery service to read the block via delivery
168-
// protocol from the orderers
169-
func (d *DeliverService) Start() {
170-
go d.checkLeaderAndRunDeliver()
171-
}
172-
173147
// Stop all service and release resources
174148
func (d *DeliverService) Stop() {
175149
d.stopDeliver()
176-
d.stateProvider.Stop()
177-
d.gossip.Stop()
178150
}
179151

180-
func (d *DeliverService) checkLeaderAndRunDeliver() {
152+
func (d *DeliverService) JoinChannel(committer committer.Committer, configBlock *common.Block) {
153+
if err := service.GetGossipService().JoinChannel(committer, configBlock); err != nil {
154+
panic("Cannot join channel, exiting")
155+
}
156+
157+
go d.checkLeaderAndRunDeliver(committer)
158+
}
181159

160+
func (d *DeliverService) checkLeaderAndRunDeliver(committer committer.Committer) {
182161
isLeader := viper.GetBool("peer.gossip.orgLeader")
183162

184163
if isLeader {
185-
d.startDeliver()
164+
d.startDeliver(committer)
186165
}
187166
}
188167

@@ -192,7 +171,7 @@ func (d *DeliverService) seekOldest() error {
192171
Seek: &orderer.SeekInfo{
193172
Start: orderer.SeekInfo_OLDEST,
194173
WindowSize: d.windowSize,
195-
ChainID: util.GetTestChainID(),
174+
ChainID: d.chainID,
196175
},
197176
},
198177
})
@@ -205,7 +184,7 @@ func (d *DeliverService) seekLatestFromCommitter(height uint64) error {
205184
Start: orderer.SeekInfo_SPECIFIED,
206185
WindowSize: d.windowSize,
207186
SpecifiedNumber: height,
208-
ChainID: util.GetTestChainID(),
187+
ChainID: d.chainID,
209188
},
210189
},
211190
})
@@ -317,17 +296,17 @@ func (d *DeliverService) readUntilClose() {
317296
}
318297
}
319298

320-
numberOfPeers := len(d.gossip.GetPeers())
299+
numberOfPeers := len(service.GetGossipService().GetPeers())
321300
// Create payload with a block received
322301
payload := createPayload(seqNum, block)
323302
// Use payload to create gossip message
324303
gossipMsg := createGossipMsg(payload)
325304
logger.Debugf("Adding payload locally, buffer seqNum = [%d], peers number [%d]", seqNum, numberOfPeers)
326305
// Add payload to local state payloads buffer
327-
d.stateProvider.AddPayload(payload)
306+
service.GetGossipService().AddPayload(d.chainID, payload)
328307
// Gossip messages with other nodes
329308
logger.Debugf("Gossiping block [%d], peers number [%d]", seqNum, numberOfPeers)
330-
d.gossip.Gossip(gossipMsg)
309+
service.GetGossipService().Gossip(gossipMsg)
331310
if err = producer.SendProducerBlockEvent(block); err != nil {
332311
logger.Errorf("Error sending block event %s", err)
333312
}

gossip/service/gossip_service.go

+145
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,145 @@
1+
/*
2+
Copyright IBM Corp. 2016 All Rights Reserved.
3+
4+
Licensed under the Apache License, Version 2.0 (the "License");
5+
you may not use this file except in compliance with the License.
6+
You may obtain a copy of the License at
7+
8+
http://www.apache.org/licenses/LICENSE-2.0
9+
10+
Unless required by applicable law or agreed to in writing, software
11+
distributed under the License is distributed on an "AS IS" BASIS,
12+
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
See the License for the specific language governing permissions and
14+
limitations under the License.
15+
*/
16+
17+
package service
18+
19+
import (
20+
"sync"
21+
"fmt"
22+
23+
pb "github.com/golang/protobuf/proto"
24+
"github.com/hyperledger/fabric/core/committer"
25+
"github.com/hyperledger/fabric/gossip/comm"
26+
gossipCommon "github.com/hyperledger/fabric/gossip/common"
27+
"github.com/hyperledger/fabric/gossip/discovery"
28+
"github.com/hyperledger/fabric/gossip/gossip"
29+
"github.com/hyperledger/fabric/gossip/integration"
30+
"github.com/hyperledger/fabric/gossip/proto"
31+
"github.com/hyperledger/fabric/gossip/state"
32+
"github.com/hyperledger/fabric/protos/common"
33+
"google.golang.org/grpc"
34+
)
35+
36+
var (
37+
gossipServiceInstance *gossipServiceImpl
38+
once sync.Once
39+
)
40+
41+
// GossipService encapsulates gossip and state capabilities into single interface
42+
type GossipService interface {
43+
gossip.Gossip
44+
45+
// JoinChannel joins new chain given the configuration block and initialized committer service
46+
JoinChannel(committer committer.Committer, block *common.Block) error
47+
// GetBlock returns block for given chain
48+
GetBlock(chainID string, index uint64) *common.Block
49+
// AddPayload appends message payload to for given chain
50+
AddPayload(chainID string, payload *proto.Payload) error
51+
}
52+
53+
type gossipServiceImpl struct {
54+
comm comm.Comm
55+
gossip gossip.Gossip
56+
chains map[string]state.GossipStateProvider
57+
lock sync.RWMutex
58+
}
59+
60+
// InitGossipService initialize gossip service
61+
func InitGossipService(endpoint string, s *grpc.Server, bootPeers ...string) {
62+
once.Do(func() {
63+
gossip, communication := integration.NewGossipComponent(endpoint, s, bootPeers...)
64+
gossipServiceInstance = &gossipServiceImpl{
65+
gossip: gossip,
66+
comm: communication,
67+
chains: make(map[string]state.GossipStateProvider),
68+
}
69+
})
70+
}
71+
72+
// GetGossipService returns an instance of gossip service
73+
func GetGossipService() GossipService {
74+
return gossipServiceInstance
75+
}
76+
77+
// JoinChannel joins the channel and initialize gossip state with given committer
78+
func (g *gossipServiceImpl) JoinChannel(commiter committer.Committer, block *common.Block) error {
79+
g.lock.Lock()
80+
defer g.lock.Unlock()
81+
82+
if block.Data == nil || block.Data.Data == nil || len(block.Data.Data) == 0 {
83+
return fmt.Errorf("Cannot join channel, configuration block is empty")
84+
}
85+
86+
envelope := &common.Envelope{}
87+
if err := pb.Unmarshal(block.Data.Data[0], envelope); err != nil {
88+
return err
89+
}
90+
91+
payload := &common.Payload{}
92+
if err := pb.Unmarshal(envelope.Payload, payload); err != nil {
93+
return err
94+
}
95+
96+
chainID := payload.Header.ChainHeader.ChainID
97+
if len(chainID) == 0 {
98+
return fmt.Errorf("Cannot join channel, with empty chainID")
99+
}
100+
// Initialize new state provider for given committer
101+
g.chains[chainID] = state.NewGossipStateProvider(g.gossip, g.comm, commiter)
102+
return nil
103+
}
104+
105+
// GetPeers returns a mapping of endpoint --> []discovery.NetworkMember
106+
func (g *gossipServiceImpl) GetPeers() []discovery.NetworkMember {
107+
return g.gossip.GetPeers()
108+
}
109+
110+
// UpdateMetadata updates the self metadata of the discovery layer
111+
func (g *gossipServiceImpl) UpdateMetadata(data []byte) {
112+
g.gossip.UpdateMetadata(data)
113+
}
114+
115+
// Gossip sends a message to other peers to the network
116+
func (g *gossipServiceImpl) Gossip(msg *proto.GossipMessage) {
117+
g.gossip.Gossip(msg)
118+
}
119+
120+
// Accept returns a channel that outputs messages from other peers
121+
func (g *gossipServiceImpl) Accept(acceptor gossipCommon.MessageAcceptor) <-chan *proto.GossipMessage {
122+
return g.gossip.Accept(acceptor)
123+
}
124+
125+
// GetBlock returns block for given chain
126+
func (g *gossipServiceImpl) GetBlock(chainID string, index uint64) *common.Block {
127+
g.lock.RLock()
128+
defer g.lock.RUnlock()
129+
return g.chains[chainID].GetBlock(index)
130+
}
131+
132+
// AddPayload appends message payload to for given chain
133+
func (g *gossipServiceImpl) AddPayload(chainID string, payload *proto.Payload) error {
134+
g.lock.RLock()
135+
defer g.lock.RUnlock()
136+
return g.chains[chainID].AddPayload(payload)
137+
}
138+
139+
// Stop stops the gossip component
140+
func (g *gossipServiceImpl) Stop() {
141+
for _, ch := range g.chains {
142+
ch.Stop()
143+
}
144+
g.gossip.Stop()
145+
}

gossip/service/gossip_service_test.go

+56
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
1+
/*
2+
Copyright IBM Corp. 2016 All Rights Reserved.
3+
4+
Licensed under the Apache License, Version 2.0 (the "License");
5+
you may not use this file except in compliance with the License.
6+
You may obtain a copy of the License at
7+
8+
http://www.apache.org/licenses/LICENSE-2.0
9+
10+
Unless required by applicable law or agreed to in writing, software
11+
distributed under the License is distributed on an "AS IS" BASIS,
12+
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
See the License for the specific language governing permissions and
14+
limitations under the License.
15+
*/
16+
17+
package service
18+
19+
import (
20+
"fmt"
21+
"net"
22+
"sync"
23+
"testing"
24+
25+
"github.com/stretchr/testify/assert"
26+
"google.golang.org/grpc"
27+
)
28+
29+
func TestInitGossipService(t *testing.T) {
30+
// Test whenever gossip service is indeed singleton
31+
grpcServer := grpc.NewServer()
32+
socket, error := net.Listen("tcp", fmt.Sprintf("%s:%d", "", 5611))
33+
assert.NoError(t, error)
34+
35+
go grpcServer.Serve(socket)
36+
defer grpcServer.Stop()
37+
38+
wg := sync.WaitGroup{}
39+
wg.Add(10)
40+
for i := 0; i < 10; i++ {
41+
go func() {
42+
InitGossipService("localhost:5611", grpcServer)
43+
wg.Done()
44+
}()
45+
}
46+
wg.Wait()
47+
48+
defer GetGossipService().Stop()
49+
gossip := GetGossipService()
50+
51+
for i := 0; i < 10; i++ {
52+
go func(gossipInstance GossipService) {
53+
assert.Equal(t, gossip, GetGossipService())
54+
}(gossip)
55+
}
56+
}

peer/common/ordererclient.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -56,12 +56,12 @@ func GetBroadcastClient() (BroadcastClient, error) {
5656

5757
conn, err := grpc.Dial(orderer, opts...)
5858
if err != nil {
59-
return nil, fmt.Errorf("Error connecting: %s", err)
59+
return nil, fmt.Errorf("Error connecting to %s due to %s", orderer, err)
6060
}
6161
client, err := ab.NewAtomicBroadcastClient(conn).Broadcast(context.TODO())
6262
if err != nil {
6363
conn.Close()
64-
return nil, fmt.Errorf("Error connecting: %s", err)
64+
return nil, fmt.Errorf("Error connecting to %s due to %s", orderer, err)
6565
}
6666

6767
return &broadcastClient{conn: conn, client: client}, nil

0 commit comments

Comments
 (0)