Skip to content

Commit

Permalink
Merge "[FAB-1623] Add restart support to Kafka orderer"
Browse files Browse the repository at this point in the history
  • Loading branch information
JonathanLevi authored and Gerrit Code Review committed Jan 24, 2017
2 parents d40e10c + 2f0aa7d commit 230f3cc
Show file tree
Hide file tree
Showing 6 changed files with 219 additions and 72 deletions.
56 changes: 35 additions & 21 deletions orderer/kafka/orderer.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,19 @@ type consenterImpl struct {
// Implements the multichain.Consenter interface. Called by multichain.newChainSupport(), which
// is itself called by multichain.NewManagerImpl() when ranging over the ledgerFactory's existingChains.
func (co *consenterImpl) HandleChain(cs multichain.ConsenterSupport, metadata *cb.Metadata) (multichain.Chain, error) {
return newChain(co, cs), nil
return newChain(co, cs, getLastOffsetPersisted(metadata)), nil
}

func getLastOffsetPersisted(metadata *cb.Metadata) int64 {
if metadata.Value != nil {
// Extract orderer-related metadata from the tip of the ledger first
kafkaMetadata := &ab.KafkaMetadata{}
if err := proto.Unmarshal(metadata.Value, kafkaMetadata); err != nil {
panic("Ledger may be corrupted: cannot unmarshal orderer metadata in most recent block")
}
return kafkaMetadata.LastOffsetPersisted
}
return (sarama.OffsetOldest - 1) // default
}

// When testing we need to inject our own broker/producer/consumer.
Expand All @@ -90,18 +102,19 @@ func (co *consenterImpl) HandleChain(cs multichain.ConsenterSupport, metadata *c
// definition of an interface (see testableConsenter below) that will
// be satisfied by both the actual and the mock object and will allow
// us to retrieve these constructors.
func newChain(consenter testableConsenter, support multichain.ConsenterSupport) *chainImpl {
func newChain(consenter testableConsenter, support multichain.ConsenterSupport, lastOffsetPersisted int64) *chainImpl {
logger.Debug("Starting chain with last persisted offset:", lastOffsetPersisted)
return &chainImpl{
consenter: consenter,
support: support,
partition: newChainPartition(support.ChainID(), rawPartition),
batchTimeout: support.SharedConfig().BatchTimeout(),
lastProcessed: sarama.OffsetOldest - 1, // TODO This should be retrieved by ConsenterSupport; also see note in loop() below
producer: consenter.prodFunc()(support.SharedConfig().KafkaBrokers(), consenter.kafkaVersion(), consenter.retryOptions()),
halted: false, // Redundant as the default value for booleans is false but added for readability
exitChan: make(chan struct{}),
haltedChan: make(chan struct{}),
setupChan: make(chan struct{}),
consenter: consenter,
support: support,
partition: newChainPartition(support.ChainID(), rawPartition),
batchTimeout: support.SharedConfig().BatchTimeout(),
lastOffsetPersisted: lastOffsetPersisted,
producer: consenter.prodFunc()(support.SharedConfig().KafkaBrokers(), consenter.kafkaVersion(), consenter.retryOptions()),
halted: false, // Redundant as the default value for booleans is false but added for readability
exitChan: make(chan struct{}),
haltedChan: make(chan struct{}),
setupChan: make(chan struct{}),
}
}

Expand All @@ -125,10 +138,10 @@ type chainImpl struct {
consenter testableConsenter
support multichain.ConsenterSupport

partition ChainPartition
batchTimeout time.Duration
lastProcessed int64
lastCutBlock uint64
partition ChainPartition
batchTimeout time.Duration
lastOffsetPersisted int64
lastCutBlock uint64

producer Producer
consumer Consumer
Expand Down Expand Up @@ -156,9 +169,7 @@ func (ch *chainImpl) Start() {
}

// 2. Set up the listener/consumer for this partition.
// TODO When restart support gets added to the common components level, start
// the consumer from lastProcessed. For now, hard-code to oldest available.
consumer, err := ch.consenter.consFunc()(ch.support.SharedConfig().KafkaBrokers(), ch.consenter.kafkaVersion(), ch.partition, ch.lastProcessed+1)
consumer, err := ch.consenter.consFunc()(ch.support.SharedConfig().KafkaBrokers(), ch.consenter.kafkaVersion(), ch.partition, ch.lastOffsetPersisted+1)
if err != nil {
logger.Criticalf("Cannot retrieve required offset from Kafka cluster for chain %s: %s", ch.partition, err)
close(ch.exitChan)
Expand Down Expand Up @@ -206,6 +217,7 @@ func (ch *chainImpl) loop() {
msg := new(ab.KafkaMessage)
var timer <-chan time.Time
var ttcNumber uint64
var encodedLastOffsetPersisted []byte

defer close(ch.haltedChan)
defer ch.producer.Close()
Expand Down Expand Up @@ -237,7 +249,8 @@ func (ch *chainImpl) loop() {
return
}
block := ch.support.CreateNextBlock(batch)
ch.support.WriteBlock(block, committers, nil)
encodedLastOffsetPersisted = utils.MarshalOrPanic(&ab.KafkaMetadata{LastOffsetPersisted: in.Offset})
ch.support.WriteBlock(block, committers, encodedLastOffsetPersisted)
ch.lastCutBlock++
logger.Debug("Proper time-to-cut received, just cut block", ch.lastCutBlock)
continue
Expand All @@ -264,7 +277,8 @@ func (ch *chainImpl) loop() {
// If !ok, batches == nil, so this will be skipped
for i, batch := range batches {
block := ch.support.CreateNextBlock(batch)
ch.support.WriteBlock(block, committers[i], nil)
encodedLastOffsetPersisted = utils.MarshalOrPanic(&ab.KafkaMetadata{LastOffsetPersisted: in.Offset})
ch.support.WriteBlock(block, committers[i], encodedLastOffsetPersisted)
ch.lastCutBlock++
logger.Debug("Batch filled, just cut block", ch.lastCutBlock)
}
Expand Down
157 changes: 124 additions & 33 deletions orderer/kafka/orderer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ limitations under the License.
package kafka

import (
"fmt"
"sync"
"testing"
"time"
Expand All @@ -39,30 +40,29 @@ func newMockSharedConfigManager() *mocksharedconfig.Manager {
return &mocksharedconfig.Manager{KafkaBrokersVal: testConf.Kafka.Brokers}
}

func syncQueueMessage(msg *cb.Envelope, chain multichain.Chain, bc *mockblockcutter.Receiver) {
chain.Enqueue(msg)
bc.Block <- struct{}{}
}

type mockConsenterImpl struct {
consenterImpl
prodDisk, consDisk chan *ab.KafkaMessage
consumerSetUp bool
t *testing.T
}

func mockNewConsenter(t *testing.T, kafkaVersion sarama.KafkaVersion, retryOptions config.Retry) *mockConsenterImpl {
func mockNewConsenter(t *testing.T, kafkaVersion sarama.KafkaVersion, retryOptions config.Retry, nextProducedOffset int64) *mockConsenterImpl {
prodDisk := make(chan *ab.KafkaMessage)
consDisk := make(chan *ab.KafkaMessage)

mockBfValue := func(brokers []string, cp ChainPartition) (Broker, error) {
return mockNewBroker(t, cp)
}
mockPfValue := func(brokers []string, kafkaVersion sarama.KafkaVersion, retryOptions config.Retry) Producer {
return mockNewProducer(t, cp, testOldestOffset, prodDisk)
// The first Send on this producer will return a blob with offset #nextProducedOffset
return mockNewProducer(t, cp, nextProducedOffset, prodDisk)
}
mockCfValue := func(brokers []string, kafkaVersion sarama.KafkaVersion, cp ChainPartition, offset int64) (Consumer, error) {
return mockNewConsumer(t, cp, offset, consDisk)
mockCfValue := func(brokers []string, kafkaVersion sarama.KafkaVersion, cp ChainPartition, lastPersistedOffset int64) (Consumer, error) {
if lastPersistedOffset != nextProducedOffset {
panic(fmt.Errorf("Mock objects about to be set up incorrectly (consumer to seek to %d, producer to post %d)", lastPersistedOffset, nextProducedOffset))
}
return mockNewConsumer(t, cp, lastPersistedOffset, consDisk)
}

return &mockConsenterImpl{
Expand Down Expand Up @@ -96,6 +96,11 @@ func prepareMockObjectDisks(t *testing.T, co *mockConsenterImpl, ch *chainImpl)
}
}

func syncQueueMessage(msg *cb.Envelope, chain multichain.Chain, bc *mockblockcutter.Receiver) {
chain.Enqueue(msg)
bc.Block <- struct{}{}
}

func waitableSyncQueueMessage(env *cb.Envelope, messagesToPickUp int, wg *sync.WaitGroup,
co *mockConsenterImpl, cs *mockmultichain.ConsenterSupport, ch *chainImpl) {
wg.Add(1)
Expand Down Expand Up @@ -128,9 +133,10 @@ func TestKafkaConsenterEmptyBatch(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -162,9 +168,10 @@ func TestKafkaConsenterBatchTimer(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -213,9 +220,10 @@ func TestKafkaConsenterTimerHaltOnFilledBatch(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -272,9 +280,10 @@ func TestKafkaConsenterConfigStyleMultiBatch(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -321,9 +330,10 @@ func TestKafkaConsenterTimeToCutForced(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -377,9 +387,10 @@ func TestKafkaConsenterTimeToCutDuplicate(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -465,9 +476,10 @@ func TestKafkaConsenterTimeToCutStale(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -523,9 +535,10 @@ func TestKafkaConsenterTimeToCutLarger(t *testing.T) {
}
defer close(cs.BlockCutterVal.Block)

co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry)
ch := newChain(co, cs)
ch.lastProcessed = testOldestOffset - 1
lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()
Expand Down Expand Up @@ -574,3 +587,81 @@ func TestKafkaConsenterTimeToCutLarger(t *testing.T) {
case <-ch.haltedChan: // If we're here, we definitely had a chance to invoke Append but didn't (which is great)
}
}

func TestGetLastOffsetPersistedEmpty(t *testing.T) {
expected := sarama.OffsetOldest - 1
actual := getLastOffsetPersisted(&cb.Metadata{})
if actual != expected {
t.Fatalf("Expected last offset %d, got %d", expected, actual)
}
}

func TestGetLastOffsetPersistedRight(t *testing.T) {
expected := int64(100)
actual := getLastOffsetPersisted(&cb.Metadata{Value: utils.MarshalOrPanic(&ab.KafkaMetadata{LastOffsetPersisted: expected})})
if actual != expected {
t.Fatalf("Expected last offset %d, got %d", expected, actual)
}
}

func TestKafkaConsenterRestart(t *testing.T) {
var wg sync.WaitGroup
defer wg.Wait()

batchTimeout, _ := time.ParseDuration("1ms")
cs := &mockmultichain.ConsenterSupport{
Batches: make(chan []*cb.Envelope),
BlockCutterVal: mockblockcutter.NewReceiver(),
ChainIDVal: provisional.TestChainID,
SharedConfigVal: &mocksharedconfig.Manager{BatchTimeoutVal: batchTimeout},
}
defer close(cs.BlockCutterVal.Block)

lastPersistedOffset := testOldestOffset - 1
nextProducedOffset := lastPersistedOffset + 1
co := mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch := newChain(co, cs, lastPersistedOffset)

go ch.Start()
defer ch.Halt()

prepareMockObjectDisks(t, co, ch)

// The second message that will be picked up is the time-to-cut message
// that will be posted when the short timer expires
waitableSyncQueueMessage(newTestEnvelope("one"), 2, &wg, co, cs, ch)

select {
case <-cs.Batches: // This is the success path
case <-time.After(testTimePadding):
t.Fatal("Expected block to be cut because batch timer expired")
}

// Stop the loop
ch.Halt()

select {
case <-cs.Batches:
t.Fatal("Expected no invocations of Append")
case <-ch.haltedChan: // If we're here, we definitely had a chance to invoke Append but didn't (which is great)
}

lastBlock := cs.WriteBlockVal
metadata, err := utils.GetMetadataFromBlock(lastBlock, cb.BlockMetadataIndex_ORDERER)
if err != nil {
logger.Fatalf("Error extracting orderer metadata for chain %x: %s", cs.ChainIDVal, err)
}

lastPersistedOffset = getLastOffsetPersisted(metadata)
nextProducedOffset = lastPersistedOffset + 1

co = mockNewConsenter(t, testConf.Kafka.Version, testConf.Kafka.Retry, nextProducedOffset)
ch = newChain(co, cs, lastPersistedOffset)
go ch.Start()
prepareMockObjectDisks(t, co, ch)

actual := ch.producer.(*mockProducerImpl).producedOffset
if actual != nextProducedOffset {
t.Fatalf("Restarted orderer post-connect should have been at offset %d, got %d instead", nextProducedOffset, actual)
}
}
1 change: 1 addition & 0 deletions orderer/multichain/chainsupport.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,7 @@ func newChainSupport(
if err != nil {
logger.Fatalf("Error extracting orderer metadata for chain %x: %s", cs.ChainID(), err)
}
logger.Debugf("Retrieved metadata for tip of chain (block #%d): %+v", cs.Reader().Height()-1, metadata)

cs.chain, err = consenter.HandleChain(cs, metadata)
if err != nil {
Expand Down
1 change: 1 addition & 0 deletions protos/orderer/ab.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

0 comments on commit 230f3cc

Please sign in to comment.