aboutsummaryrefslogtreecommitdiff
path: root/weed/mq/client/publish_stream_processor.go
blob: c23c6a64a3ab67a1f25e62e5f33e74d1fa38a844 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
package client

import (
	"context"
	flatbuffers "github.com/google/flatbuffers/go"
	"github.com/seaweedfs/seaweedfs/weed/mq/segment"
	"github.com/seaweedfs/seaweedfs/weed/pb"
	"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
	"github.com/seaweedfs/seaweedfs/weed/util"
	"google.golang.org/grpc"
	"google.golang.org/grpc/credentials/insecure"
	"log"
	"sync"
	"sync/atomic"
	"time"
)

const (
	batchCountLimit = 3
)

type PublishStreamProcessor struct {
	// attributes
	ProducerId     int32
	ProducerEpoch  int32
	grpcDialOption grpc.DialOption

	// input
	sync.Mutex

	timeout time.Duration

	// convert into bytes
	messagesChan           chan *Message
	builders               chan *flatbuffers.Builder
	batchMessageCountLimit int

	messagesSequence int64

	// done channel
	doneChan chan struct{}
}

type UploadProcess struct {
	bufferBuilder *flatbuffers.Builder
	batchBuilder  *segment.MessageBatchBuilder
}

func NewPublishStreamProcessor(batchMessageCountLimit int, timeout time.Duration) *PublishStreamProcessor {
	t := &PublishStreamProcessor{
		grpcDialOption:         grpc.WithTransportCredentials(insecure.NewCredentials()),
		batchMessageCountLimit: batchMessageCountLimit,
		builders:               make(chan *flatbuffers.Builder, batchCountLimit),
		messagesChan:           make(chan *Message, 1024),
		doneChan:               make(chan struct{}),
		timeout:                timeout,
	}
	for i := 0; i < batchCountLimit; i++ {
		t.builders <- flatbuffers.NewBuilder(4 * 1024 * 1024)
	}
	go t.doLoopUpload()
	return t
}

func (p *PublishStreamProcessor) AddMessage(m *Message) error {
	p.messagesChan <- m
	return nil
}

func (p *PublishStreamProcessor) Shutdown() error {
	p.doneChan <- struct{}{}
	return nil
}

func (p *PublishStreamProcessor) doFlush(stream mq_pb.SeaweedMessaging_PublishMessageClient, messages []*Message) error {

	if len(messages) == 0 {
		return nil
	}

	builder := <-p.builders
	bb := segment.NewMessageBatchBuilder(builder, p.ProducerId, p.ProducerEpoch, 3, 4)
	for _, m := range messages {
		bb.AddMessage(p.messagesSequence, m.Ts.UnixNano(), m.Properties, m.Key, m.Content)
		p.messagesSequence++
	}
	bb.BuildMessageBatch()
	defer func() {
		p.builders <- builder
	}()

	return stream.Send(&mq_pb.PublishRequest{
		Data: &mq_pb.PublishRequest_DataMessage{
			Message: bb.GetBytes(),
		},
	})

}

func (p *PublishStreamProcessor) doLoopUpload() {

	brokerGrpcAddress := "localhost:17777"

	// TOOD parallelize the uploading with separate uploader
	messages := make([]*Message, 0, p.batchMessageCountLimit)

	util.RetryForever("publish message", func() error {
		return pb.WithBrokerGrpcClient(false, brokerGrpcAddress, p.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {

			ctx, cancel := context.WithCancel(context.Background())
			defer cancel()

			stream, err := client.PublishMessage(ctx)
			if err != nil {
				log.Printf("grpc PublishMessage: %v", err)
				return err
			}

			var atomicStatus int64
			go func() {
				resp, err := stream.Recv()
				if err != nil {
					log.Printf("response error: %v", err)
				} else {
					log.Printf("response: %v", resp.AckSequence)
				}
				if atomic.LoadInt64(&atomicStatus) < 0 {
					return
				}
			}()

			var flushErr error
			// retry previously failed messages
			if len(messages) >= p.batchMessageCountLimit {
				flushErr = p.doFlush(stream, messages)
				if flushErr != nil {
					return flushErr
				}
				messages = messages[:0]
			}

			for {
				select {
				case m := <-p.messagesChan:
					messages = append(messages, m)
					if len(messages) >= p.batchMessageCountLimit {
						if flushErr = p.doFlush(stream, messages); flushErr != nil {
							return flushErr
						}
						messages = messages[:0]
					}
				case <-time.After(p.timeout):
					if flushErr = p.doFlush(stream, messages); flushErr != nil {
						return flushErr
					}
					messages = messages[:0]
				case <-p.doneChan:
					if flushErr = p.doFlush(stream, messages); flushErr != nil {
						return flushErr
					}
					messages = messages[:0]
					println("$ stopping ...")
					break
				}
			}

			// stop the response consuming goroutine
			atomic.StoreInt64(&atomicStatus, -1)

			return flushErr

		})
	}, func(err error) (shouldContinue bool) {
		log.Printf("failed with grpc %s: %v", brokerGrpcAddress, err)
		return true
	})

}