blob: beda53785c266977bcc9b92c929ae33d24ac39c2 [file] [log] [blame]
William Kurkianea869482019-04-09 15:16:11 -04001/*
2 * Copyright 2018-present Open Networking Foundation
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 */
16package kafka
17
18import (
19 "context"
20 "errors"
21 "fmt"
divyadesaid26f6b12020-03-19 06:30:28 +000022 "google.golang.org/grpc/codes"
23 "google.golang.org/grpc/status"
William Kurkianea869482019-04-09 15:16:11 -040024 "reflect"
25 "strings"
26 "sync"
27 "time"
William Kurkianea869482019-04-09 15:16:11 -040028
Esin Karamanccb714b2019-11-29 15:02:06 +000029 "github.com/golang/protobuf/proto"
30 "github.com/golang/protobuf/ptypes"
31 "github.com/golang/protobuf/ptypes/any"
32 "github.com/google/uuid"
33 "github.com/opencord/voltha-lib-go/v3/pkg/log"
34 ic "github.com/opencord/voltha-protos/v3/go/inter_container"
35)
William Kurkianea869482019-04-09 15:16:11 -040036
37const (
38 DefaultMaxRetries = 3
divyadesaid26f6b12020-03-19 06:30:28 +000039 DefaultRequestTimeout = 60000 // 60000 milliseconds - to handle a wider latency range
William Kurkianea869482019-04-09 15:16:11 -040040)
41
42const (
43 TransactionKey = "transactionID"
44 FromTopic = "fromTopic"
45)
46
kdarapub26b4502019-10-05 03:02:33 +053047var ErrorTransactionNotAcquired = errors.New("transaction-not-acquired")
48var ErrorTransactionInvalidId = errors.New("transaction-invalid-id")
49
William Kurkianea869482019-04-09 15:16:11 -040050// requestHandlerChannel represents an interface associated with a channel. Whenever, an event is
51// obtained from that channel, this interface is invoked. This is used to handle
52// async requests into the Core via the kafka messaging bus
53type requestHandlerChannel struct {
54 requesthandlerInterface interface{}
55 ch <-chan *ic.InterContainerMessage
56}
57
58// transactionChannel represents a combination of a topic and a channel onto which a response received
59// on the kafka bus will be sent to
60type transactionChannel struct {
61 topic *Topic
62 ch chan *ic.InterContainerMessage
63}
64
npujarec5762e2020-01-01 14:08:48 +053065type InterContainerProxy interface {
66 Start() error
67 Stop()
68 GetDefaultTopic() *Topic
69 DeviceDiscovered(deviceId string, deviceType string, parentId string, publisher string) error
70 InvokeRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic, waitForResponse bool, key string, kvArgs ...*KVArg) (bool, *any.Any)
divyadesaid26f6b12020-03-19 06:30:28 +000071 InvokeAsyncRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic, waitForResponse bool, key string, kvArgs ...*KVArg) chan *RpcResponse
npujarec5762e2020-01-01 14:08:48 +053072 SubscribeWithRequestHandlerInterface(topic Topic, handler interface{}) error
73 SubscribeWithDefaultRequestHandler(topic Topic, initialOffset int64) error
74 UnSubscribeFromRequestHandler(topic Topic) error
75 DeleteTopic(topic Topic) error
76 EnableLivenessChannel(enable bool) chan bool
77 SendLiveness() error
78}
79
80// interContainerProxy represents the messaging proxy
81type interContainerProxy struct {
William Kurkianea869482019-04-09 15:16:11 -040082 kafkaHost string
83 kafkaPort int
npujarec5762e2020-01-01 14:08:48 +053084 defaultTopic *Topic
William Kurkianea869482019-04-09 15:16:11 -040085 defaultRequestHandlerInterface interface{}
86 deviceDiscoveryTopic *Topic
87 kafkaClient Client
npujarec5762e2020-01-01 14:08:48 +053088 doneCh chan struct{}
89 doneOnce sync.Once
William Kurkianea869482019-04-09 15:16:11 -040090
91 // This map is used to map a topic to an interface and channel. When a request is received
92 // on that channel (registered to the topic) then that interface is invoked.
93 topicToRequestHandlerChannelMap map[string]*requestHandlerChannel
94 lockTopicRequestHandlerChannelMap sync.RWMutex
95
96 // This map is used to map a channel to a response topic. This channel handles all responses on that
97 // channel for that topic and forward them to the appropriate consumers channel, using the
98 // transactionIdToChannelMap.
99 topicToResponseChannelMap map[string]<-chan *ic.InterContainerMessage
100 lockTopicResponseChannelMap sync.RWMutex
101
102 // This map is used to map a transaction to a consumers channel. This is used whenever a request has been
103 // sent out and we are waiting for a response.
104 transactionIdToChannelMap map[string]*transactionChannel
105 lockTransactionIdToChannelMap sync.RWMutex
106}
107
npujarec5762e2020-01-01 14:08:48 +0530108type InterContainerProxyOption func(*interContainerProxy)
William Kurkianea869482019-04-09 15:16:11 -0400109
110func InterContainerHost(host string) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530111 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400112 args.kafkaHost = host
113 }
114}
115
116func InterContainerPort(port int) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530117 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400118 args.kafkaPort = port
119 }
120}
121
122func DefaultTopic(topic *Topic) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530123 return func(args *interContainerProxy) {
124 args.defaultTopic = topic
William Kurkianea869482019-04-09 15:16:11 -0400125 }
126}
127
128func DeviceDiscoveryTopic(topic *Topic) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530129 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400130 args.deviceDiscoveryTopic = topic
131 }
132}
133
134func RequestHandlerInterface(handler interface{}) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530135 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400136 args.defaultRequestHandlerInterface = handler
137 }
138}
139
140func MsgClient(client Client) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530141 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400142 args.kafkaClient = client
143 }
144}
145
npujarec5762e2020-01-01 14:08:48 +0530146func newInterContainerProxy(opts ...InterContainerProxyOption) *interContainerProxy {
147 proxy := &interContainerProxy{
William Kurkianea869482019-04-09 15:16:11 -0400148 kafkaHost: DefaultKafkaHost,
149 kafkaPort: DefaultKafkaPort,
npujarec5762e2020-01-01 14:08:48 +0530150 doneCh: make(chan struct{}),
William Kurkianea869482019-04-09 15:16:11 -0400151 }
152
153 for _, option := range opts {
154 option(proxy)
155 }
156
npujarec5762e2020-01-01 14:08:48 +0530157 return proxy
William Kurkianea869482019-04-09 15:16:11 -0400158}
159
npujarec5762e2020-01-01 14:08:48 +0530160func NewInterContainerProxy(opts ...InterContainerProxyOption) InterContainerProxy {
161 return newInterContainerProxy(opts...)
162}
163
164func (kp *interContainerProxy) Start() error {
Esin Karamanccb714b2019-11-29 15:02:06 +0000165 logger.Info("Starting-Proxy")
William Kurkianea869482019-04-09 15:16:11 -0400166
167 // Kafka MsgClient should already have been created. If not, output fatal error
168 if kp.kafkaClient == nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000169 logger.Fatal("kafka-client-not-set")
William Kurkianea869482019-04-09 15:16:11 -0400170 }
171
William Kurkianea869482019-04-09 15:16:11 -0400172 // Start the kafka client
173 if err := kp.kafkaClient.Start(); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000174 logger.Errorw("Cannot-create-kafka-proxy", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400175 return err
176 }
177
178 // Create the topic to response channel map
179 kp.topicToResponseChannelMap = make(map[string]<-chan *ic.InterContainerMessage)
180 //
181 // Create the transactionId to Channel Map
182 kp.transactionIdToChannelMap = make(map[string]*transactionChannel)
183
184 // Create the topic to request channel map
185 kp.topicToRequestHandlerChannelMap = make(map[string]*requestHandlerChannel)
186
187 return nil
188}
189
npujarec5762e2020-01-01 14:08:48 +0530190func (kp *interContainerProxy) Stop() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000191 logger.Info("stopping-intercontainer-proxy")
npujarec5762e2020-01-01 14:08:48 +0530192 kp.doneOnce.Do(func() { close(kp.doneCh) })
William Kurkianea869482019-04-09 15:16:11 -0400193 // TODO : Perform cleanup
194 kp.kafkaClient.Stop()
Scott Bakere701b862020-02-20 16:19:16 -0800195 err := kp.deleteAllTopicRequestHandlerChannelMap()
196 if err != nil {
Scott Baker24f83e22020-03-30 16:14:28 -0700197 logger.Errorw("failed-delete-all-topic-request-handler-channel-map", log.Fields{"error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800198 }
199 err = kp.deleteAllTopicResponseChannelMap()
200 if err != nil {
Scott Baker24f83e22020-03-30 16:14:28 -0700201 logger.Errorw("failed-delete-all-topic-response-channel-map", log.Fields{"error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800202 }
203 kp.deleteAllTransactionIdToChannelMap()
William Kurkianea869482019-04-09 15:16:11 -0400204}
205
npujarec5762e2020-01-01 14:08:48 +0530206func (kp *interContainerProxy) GetDefaultTopic() *Topic {
207 return kp.defaultTopic
208}
209
William Kurkianea869482019-04-09 15:16:11 -0400210// DeviceDiscovered publish the discovered device onto the kafka messaging bus
npujarec5762e2020-01-01 14:08:48 +0530211func (kp *interContainerProxy) DeviceDiscovered(deviceId string, deviceType string, parentId string, publisher string) error {
Esin Karamanccb714b2019-11-29 15:02:06 +0000212 logger.Debugw("sending-device-discovery-msg", log.Fields{"deviceId": deviceId})
William Kurkianea869482019-04-09 15:16:11 -0400213 // Simple validation
214 if deviceId == "" || deviceType == "" {
Esin Karamanccb714b2019-11-29 15:02:06 +0000215 logger.Errorw("invalid-parameters", log.Fields{"id": deviceId, "type": deviceType})
William Kurkianea869482019-04-09 15:16:11 -0400216 return errors.New("invalid-parameters")
217 }
218 // Create the device discovery message
219 header := &ic.Header{
220 Id: uuid.New().String(),
221 Type: ic.MessageType_DEVICE_DISCOVERED,
npujarec5762e2020-01-01 14:08:48 +0530222 FromTopic: kp.defaultTopic.Name,
William Kurkianea869482019-04-09 15:16:11 -0400223 ToTopic: kp.deviceDiscoveryTopic.Name,
Scott Bakered4a8e72020-04-17 11:10:20 -0700224 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400225 }
226 body := &ic.DeviceDiscovered{
227 Id: deviceId,
228 DeviceType: deviceType,
229 ParentId: parentId,
230 Publisher: publisher,
231 }
232
233 var marshalledData *any.Any
234 var err error
235 if marshalledData, err = ptypes.MarshalAny(body); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000236 logger.Errorw("cannot-marshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400237 return err
238 }
239 msg := &ic.InterContainerMessage{
240 Header: header,
241 Body: marshalledData,
242 }
243
244 // Send the message
245 if err := kp.kafkaClient.Send(msg, kp.deviceDiscoveryTopic); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000246 logger.Errorw("cannot-send-device-discovery-message", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400247 return err
248 }
249 return nil
250}
251
divyadesaid26f6b12020-03-19 06:30:28 +0000252// InvokeAsyncRPC is used to make an RPC request asynchronously
253func (kp *interContainerProxy) InvokeAsyncRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
254 waitForResponse bool, key string, kvArgs ...*KVArg) chan *RpcResponse {
255
256 logger.Debugw("InvokeAsyncRPC", log.Fields{"rpc": rpc, "key": key})
257 // If a replyToTopic is provided then we use it, otherwise just use the default toTopic. The replyToTopic is
258 // typically the device ID.
259 responseTopic := replyToTopic
260 if responseTopic == nil {
261 responseTopic = kp.GetDefaultTopic()
262 }
263
264 chnl := make(chan *RpcResponse)
265
266 go func() {
267
268 // once we're done,
269 // close the response channel
270 defer close(chnl)
271
272 var err error
273 var protoRequest *ic.InterContainerMessage
274
275 // Encode the request
276 protoRequest, err = encodeRequest(rpc, toTopic, responseTopic, key, kvArgs...)
277 if err != nil {
278 logger.Warnw("cannot-format-request", log.Fields{"rpc": rpc, "error": err})
279 chnl <- NewResponse(RpcFormattingError, err, nil)
280 return
281 }
282
283 // Subscribe for response, if needed, before sending request
284 var ch <-chan *ic.InterContainerMessage
285 if ch, err = kp.subscribeForResponse(*responseTopic, protoRequest.Header.Id); err != nil {
286 logger.Errorw("failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
287 chnl <- NewResponse(RpcTransportError, err, nil)
288 return
289 }
290
291 // Send request - if the topic is formatted with a device Id then we will send the request using a
292 // specific key, hence ensuring a single partition is used to publish the request. This ensures that the
293 // subscriber on that topic will receive the request in the order it was sent. The key used is the deviceId.
294 logger.Debugw("sending-msg", log.Fields{"rpc": rpc, "toTopic": toTopic, "replyTopic": responseTopic, "key": key, "xId": protoRequest.Header.Id})
295
296 // if the message is not sent on kafka publish an event an close the channel
297 if err = kp.kafkaClient.Send(protoRequest, toTopic, key); err != nil {
298 chnl <- NewResponse(RpcTransportError, err, nil)
299 return
300 }
301
302 // if the client is not waiting for a response send the ack and close the channel
303 chnl <- NewResponse(RpcSent, nil, nil)
304 if !waitForResponse {
305 return
306 }
307
308 defer func() {
309 // Remove the subscription for a response on return
310 if err := kp.unSubscribeForResponse(protoRequest.Header.Id); err != nil {
311 logger.Warnw("invoke-async-rpc-unsubscriber-for-response-failed", log.Fields{"err": err})
312 }
313 }()
314
315 // Wait for response as well as timeout or cancellation
316 select {
317 case msg, ok := <-ch:
318 if !ok {
319 logger.Warnw("channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
320 chnl <- NewResponse(RpcTransportError, status.Error(codes.Aborted, "channel closed"), nil)
321 }
322 logger.Debugw("received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
323 if responseBody, err := decodeResponse(msg); err != nil {
324 chnl <- NewResponse(RpcReply, err, nil)
325 } else {
326 if responseBody.Success {
327 chnl <- NewResponse(RpcReply, nil, responseBody.Result)
328 } else {
329 // response body contains an error
330 unpackErr := &ic.Error{}
331 if err := ptypes.UnmarshalAny(responseBody.Result, unpackErr); err != nil {
332 chnl <- NewResponse(RpcReply, err, nil)
333 } else {
334 chnl <- NewResponse(RpcReply, status.Error(codes.Internal, unpackErr.Reason), nil)
335 }
336 }
337 }
338 case <-ctx.Done():
339 logger.Errorw("context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
340 err := status.Error(codes.DeadlineExceeded, ctx.Err().Error())
341 chnl <- NewResponse(RpcTimeout, err, nil)
342 case <-kp.doneCh:
343 chnl <- NewResponse(RpcSystemClosing, nil, nil)
344 logger.Warnw("received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
345 }
346 }()
347 return chnl
348}
349
William Kurkianea869482019-04-09 15:16:11 -0400350// InvokeRPC is used to send a request to a given topic
npujarec5762e2020-01-01 14:08:48 +0530351func (kp *interContainerProxy) InvokeRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
William Kurkianea869482019-04-09 15:16:11 -0400352 waitForResponse bool, key string, kvArgs ...*KVArg) (bool, *any.Any) {
353
354 // If a replyToTopic is provided then we use it, otherwise just use the default toTopic. The replyToTopic is
355 // typically the device ID.
356 responseTopic := replyToTopic
357 if responseTopic == nil {
npujarec5762e2020-01-01 14:08:48 +0530358 responseTopic = kp.defaultTopic
William Kurkianea869482019-04-09 15:16:11 -0400359 }
360
361 // Encode the request
362 protoRequest, err := encodeRequest(rpc, toTopic, responseTopic, key, kvArgs...)
363 if err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000364 logger.Warnw("cannot-format-request", log.Fields{"rpc": rpc, "error": err})
William Kurkianea869482019-04-09 15:16:11 -0400365 return false, nil
366 }
367
368 // Subscribe for response, if needed, before sending request
369 var ch <-chan *ic.InterContainerMessage
370 if waitForResponse {
371 var err error
372 if ch, err = kp.subscribeForResponse(*responseTopic, protoRequest.Header.Id); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000373 logger.Errorw("failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400374 }
375 }
376
377 // Send request - if the topic is formatted with a device Id then we will send the request using a
378 // specific key, hence ensuring a single partition is used to publish the request. This ensures that the
379 // subscriber on that topic will receive the request in the order it was sent. The key used is the deviceId.
380 //key := GetDeviceIdFromTopic(*toTopic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000381 logger.Debugw("sending-msg", log.Fields{"rpc": rpc, "toTopic": toTopic, "replyTopic": responseTopic, "key": key, "xId": protoRequest.Header.Id})
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000382 go func() {
383 if err := kp.kafkaClient.Send(protoRequest, toTopic, key); err != nil {
384 logger.Errorw("send-failed", log.Fields{
385 "topic": toTopic,
386 "key": key,
387 "error": err})
388 }
389 }()
William Kurkianea869482019-04-09 15:16:11 -0400390
391 if waitForResponse {
392 // Create a child context based on the parent context, if any
393 var cancel context.CancelFunc
394 childCtx := context.Background()
395 if ctx == nil {
396 ctx, cancel = context.WithTimeout(context.Background(), DefaultRequestTimeout*time.Millisecond)
397 } else {
398 childCtx, cancel = context.WithTimeout(ctx, DefaultRequestTimeout*time.Millisecond)
399 }
400 defer cancel()
401
402 // Wait for response as well as timeout or cancellation
403 // Remove the subscription for a response on return
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000404 defer func() {
405 if err := kp.unSubscribeForResponse(protoRequest.Header.Id); err != nil {
406 logger.Errorw("response-unsubscribe-failed", log.Fields{
407 "id": protoRequest.Header.Id,
408 "error": err})
409 }
410 }()
William Kurkianea869482019-04-09 15:16:11 -0400411 select {
412 case msg, ok := <-ch:
413 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000414 logger.Warnw("channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400415 protoError := &ic.Error{Reason: "channel-closed"}
416 var marshalledArg *any.Any
417 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
418 return false, nil // Should never happen
419 }
420 return false, marshalledArg
421 }
Esin Karamanccb714b2019-11-29 15:02:06 +0000422 logger.Debugw("received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400423 var responseBody *ic.InterContainerResponseBody
424 var err error
425 if responseBody, err = decodeResponse(msg); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000426 logger.Errorw("decode-response-error", log.Fields{"error": err})
npujarec5762e2020-01-01 14:08:48 +0530427 // FIXME we should return something
William Kurkianea869482019-04-09 15:16:11 -0400428 }
429 return responseBody.Success, responseBody.Result
430 case <-ctx.Done():
Esin Karamanccb714b2019-11-29 15:02:06 +0000431 logger.Debugw("context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
William Kurkianea869482019-04-09 15:16:11 -0400432 // pack the error as proto any type
npujarec5762e2020-01-01 14:08:48 +0530433 protoError := &ic.Error{Reason: ctx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
434
William Kurkianea869482019-04-09 15:16:11 -0400435 var marshalledArg *any.Any
436 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
437 return false, nil // Should never happen
438 }
439 return false, marshalledArg
440 case <-childCtx.Done():
Esin Karamanccb714b2019-11-29 15:02:06 +0000441 logger.Debugw("context-cancelled", log.Fields{"rpc": rpc, "ctx": childCtx.Err()})
William Kurkianea869482019-04-09 15:16:11 -0400442 // pack the error as proto any type
npujarec5762e2020-01-01 14:08:48 +0530443 protoError := &ic.Error{Reason: childCtx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
444
William Kurkianea869482019-04-09 15:16:11 -0400445 var marshalledArg *any.Any
446 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
447 return false, nil // Should never happen
448 }
449 return false, marshalledArg
450 case <-kp.doneCh:
Esin Karamanccb714b2019-11-29 15:02:06 +0000451 logger.Infow("received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
William Kurkianea869482019-04-09 15:16:11 -0400452 return true, nil
453 }
454 }
455 return true, nil
456}
457
458// SubscribeWithRequestHandlerInterface allows a caller to assign a target object to be invoked automatically
459// when a message is received on a given topic
npujarec5762e2020-01-01 14:08:48 +0530460func (kp *interContainerProxy) SubscribeWithRequestHandlerInterface(topic Topic, handler interface{}) error {
William Kurkianea869482019-04-09 15:16:11 -0400461
462 // Subscribe to receive messages for that topic
463 var ch <-chan *ic.InterContainerMessage
464 var err error
465 if ch, err = kp.kafkaClient.Subscribe(&topic); err != nil {
466 //if ch, err = kp.Subscribe(topic); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000467 logger.Errorw("failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
Matt Jeannereteb5059f2019-07-19 06:11:00 -0400468 return err
William Kurkianea869482019-04-09 15:16:11 -0400469 }
470
471 kp.defaultRequestHandlerInterface = handler
472 kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: handler, ch: ch})
473 // Launch a go routine to receive and process kafka messages
474 go kp.waitForMessages(ch, topic, handler)
475
476 return nil
477}
478
479// SubscribeWithDefaultRequestHandler allows a caller to add a topic to an existing target object to be invoked automatically
480// when a message is received on a given topic. So far there is only 1 target registered per microservice
npujarec5762e2020-01-01 14:08:48 +0530481func (kp *interContainerProxy) SubscribeWithDefaultRequestHandler(topic Topic, initialOffset int64) error {
William Kurkianea869482019-04-09 15:16:11 -0400482 // Subscribe to receive messages for that topic
483 var ch <-chan *ic.InterContainerMessage
484 var err error
485 if ch, err = kp.kafkaClient.Subscribe(&topic, &KVArg{Key: Offset, Value: initialOffset}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000486 logger.Errorw("failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400487 return err
488 }
489 kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: kp.defaultRequestHandlerInterface, ch: ch})
490
491 // Launch a go routine to receive and process kafka messages
492 go kp.waitForMessages(ch, topic, kp.defaultRequestHandlerInterface)
493
494 return nil
495}
496
npujarec5762e2020-01-01 14:08:48 +0530497func (kp *interContainerProxy) UnSubscribeFromRequestHandler(topic Topic) error {
William Kurkianea869482019-04-09 15:16:11 -0400498 return kp.deleteFromTopicRequestHandlerChannelMap(topic.Name)
499}
500
npujarec5762e2020-01-01 14:08:48 +0530501func (kp *interContainerProxy) deleteFromTopicResponseChannelMap(topic string) error {
William Kurkianea869482019-04-09 15:16:11 -0400502 kp.lockTopicResponseChannelMap.Lock()
503 defer kp.lockTopicResponseChannelMap.Unlock()
504 if _, exist := kp.topicToResponseChannelMap[topic]; exist {
505 // Unsubscribe to this topic first - this will close the subscribed channel
506 var err error
507 if err = kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000508 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic})
William Kurkianea869482019-04-09 15:16:11 -0400509 }
510 delete(kp.topicToResponseChannelMap, topic)
511 return err
512 } else {
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000513 return fmt.Errorf("%s-Topic-not-found", topic)
William Kurkianea869482019-04-09 15:16:11 -0400514 }
515}
516
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000517// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530518func (kp *interContainerProxy) deleteAllTopicResponseChannelMap() error {
Scott Bakere701b862020-02-20 16:19:16 -0800519 logger.Debug("delete-all-topic-response-channel")
William Kurkianea869482019-04-09 15:16:11 -0400520 kp.lockTopicResponseChannelMap.Lock()
521 defer kp.lockTopicResponseChannelMap.Unlock()
Scott Bakere701b862020-02-20 16:19:16 -0800522 var unsubscribeFailTopics []string
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000523 for topic := range kp.topicToResponseChannelMap {
William Kurkianea869482019-04-09 15:16:11 -0400524 // Unsubscribe to this topic first - this will close the subscribed channel
Scott Bakere701b862020-02-20 16:19:16 -0800525 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
526 unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000527 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic, "error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800528 // Do not return. Continue to try to unsubscribe to other topics.
529 } else {
530 // Only delete from channel map if successfully unsubscribed.
531 delete(kp.topicToResponseChannelMap, topic)
William Kurkianea869482019-04-09 15:16:11 -0400532 }
William Kurkianea869482019-04-09 15:16:11 -0400533 }
Scott Bakere701b862020-02-20 16:19:16 -0800534 if len(unsubscribeFailTopics) > 0 {
535 return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
536 }
537 return nil
William Kurkianea869482019-04-09 15:16:11 -0400538}
539
npujarec5762e2020-01-01 14:08:48 +0530540func (kp *interContainerProxy) addToTopicRequestHandlerChannelMap(topic string, arg *requestHandlerChannel) {
William Kurkianea869482019-04-09 15:16:11 -0400541 kp.lockTopicRequestHandlerChannelMap.Lock()
542 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
543 if _, exist := kp.topicToRequestHandlerChannelMap[topic]; !exist {
544 kp.topicToRequestHandlerChannelMap[topic] = arg
545 }
546}
547
npujarec5762e2020-01-01 14:08:48 +0530548func (kp *interContainerProxy) deleteFromTopicRequestHandlerChannelMap(topic string) error {
William Kurkianea869482019-04-09 15:16:11 -0400549 kp.lockTopicRequestHandlerChannelMap.Lock()
550 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
551 if _, exist := kp.topicToRequestHandlerChannelMap[topic]; exist {
552 // Close the kafka client client first by unsubscribing to this topic
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000553 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
554 return err
555 }
William Kurkianea869482019-04-09 15:16:11 -0400556 delete(kp.topicToRequestHandlerChannelMap, topic)
557 return nil
558 } else {
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000559 return fmt.Errorf("%s-Topic-not-found", topic)
William Kurkianea869482019-04-09 15:16:11 -0400560 }
561}
562
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000563// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530564func (kp *interContainerProxy) deleteAllTopicRequestHandlerChannelMap() error {
Scott Bakere701b862020-02-20 16:19:16 -0800565 logger.Debug("delete-all-topic-request-channel")
William Kurkianea869482019-04-09 15:16:11 -0400566 kp.lockTopicRequestHandlerChannelMap.Lock()
567 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
Scott Bakere701b862020-02-20 16:19:16 -0800568 var unsubscribeFailTopics []string
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000569 for topic := range kp.topicToRequestHandlerChannelMap {
William Kurkianea869482019-04-09 15:16:11 -0400570 // Close the kafka client client first by unsubscribing to this topic
Scott Bakere701b862020-02-20 16:19:16 -0800571 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
572 unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000573 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic, "error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800574 // Do not return. Continue to try to unsubscribe to other topics.
575 } else {
576 // Only delete from channel map if successfully unsubscribed.
577 delete(kp.topicToRequestHandlerChannelMap, topic)
William Kurkianea869482019-04-09 15:16:11 -0400578 }
William Kurkianea869482019-04-09 15:16:11 -0400579 }
Scott Bakere701b862020-02-20 16:19:16 -0800580 if len(unsubscribeFailTopics) > 0 {
581 return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
582 }
583 return nil
William Kurkianea869482019-04-09 15:16:11 -0400584}
585
npujarec5762e2020-01-01 14:08:48 +0530586func (kp *interContainerProxy) addToTransactionIdToChannelMap(id string, topic *Topic, arg chan *ic.InterContainerMessage) {
William Kurkianea869482019-04-09 15:16:11 -0400587 kp.lockTransactionIdToChannelMap.Lock()
588 defer kp.lockTransactionIdToChannelMap.Unlock()
589 if _, exist := kp.transactionIdToChannelMap[id]; !exist {
590 kp.transactionIdToChannelMap[id] = &transactionChannel{topic: topic, ch: arg}
591 }
592}
593
npujarec5762e2020-01-01 14:08:48 +0530594func (kp *interContainerProxy) deleteFromTransactionIdToChannelMap(id string) {
William Kurkianea869482019-04-09 15:16:11 -0400595 kp.lockTransactionIdToChannelMap.Lock()
596 defer kp.lockTransactionIdToChannelMap.Unlock()
597 if transChannel, exist := kp.transactionIdToChannelMap[id]; exist {
598 // Close the channel first
599 close(transChannel.ch)
600 delete(kp.transactionIdToChannelMap, id)
601 }
602}
603
npujarec5762e2020-01-01 14:08:48 +0530604func (kp *interContainerProxy) deleteTopicTransactionIdToChannelMap(id string) {
William Kurkianea869482019-04-09 15:16:11 -0400605 kp.lockTransactionIdToChannelMap.Lock()
606 defer kp.lockTransactionIdToChannelMap.Unlock()
607 for key, value := range kp.transactionIdToChannelMap {
608 if value.topic.Name == id {
609 close(value.ch)
610 delete(kp.transactionIdToChannelMap, key)
611 }
612 }
613}
614
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000615// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530616func (kp *interContainerProxy) deleteAllTransactionIdToChannelMap() {
Scott Bakere701b862020-02-20 16:19:16 -0800617 logger.Debug("delete-all-transaction-id-channel-map")
William Kurkianea869482019-04-09 15:16:11 -0400618 kp.lockTransactionIdToChannelMap.Lock()
619 defer kp.lockTransactionIdToChannelMap.Unlock()
620 for key, value := range kp.transactionIdToChannelMap {
621 close(value.ch)
622 delete(kp.transactionIdToChannelMap, key)
623 }
624}
625
npujarec5762e2020-01-01 14:08:48 +0530626func (kp *interContainerProxy) DeleteTopic(topic Topic) error {
William Kurkianea869482019-04-09 15:16:11 -0400627 // If we have any consumers on that topic we need to close them
628 if err := kp.deleteFromTopicResponseChannelMap(topic.Name); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000629 logger.Errorw("delete-from-topic-responsechannelmap-failed", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400630 }
631 if err := kp.deleteFromTopicRequestHandlerChannelMap(topic.Name); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000632 logger.Errorw("delete-from-topic-requesthandlerchannelmap-failed", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400633 }
634 kp.deleteTopicTransactionIdToChannelMap(topic.Name)
635
636 return kp.kafkaClient.DeleteTopic(&topic)
637}
638
639func encodeReturnedValue(returnedVal interface{}) (*any.Any, error) {
640 // Encode the response argument - needs to be a proto message
641 if returnedVal == nil {
642 return nil, nil
643 }
644 protoValue, ok := returnedVal.(proto.Message)
645 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000646 logger.Warnw("response-value-not-proto-message", log.Fields{"error": ok, "returnVal": returnedVal})
William Kurkianea869482019-04-09 15:16:11 -0400647 err := errors.New("response-value-not-proto-message")
648 return nil, err
649 }
650
651 // Marshal the returned value, if any
652 var marshalledReturnedVal *any.Any
653 var err error
654 if marshalledReturnedVal, err = ptypes.MarshalAny(protoValue); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000655 logger.Warnw("cannot-marshal-returned-val", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400656 return nil, err
657 }
658 return marshalledReturnedVal, nil
659}
660
661func encodeDefaultFailedResponse(request *ic.InterContainerMessage) *ic.InterContainerMessage {
662 responseHeader := &ic.Header{
663 Id: request.Header.Id,
664 Type: ic.MessageType_RESPONSE,
665 FromTopic: request.Header.ToTopic,
666 ToTopic: request.Header.FromTopic,
Scott Bakered4a8e72020-04-17 11:10:20 -0700667 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400668 }
669 responseBody := &ic.InterContainerResponseBody{
670 Success: false,
671 Result: nil,
672 }
673 var marshalledResponseBody *any.Any
674 var err error
675 // Error should never happen here
676 if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000677 logger.Warnw("cannot-marshal-failed-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400678 }
679
680 return &ic.InterContainerMessage{
681 Header: responseHeader,
682 Body: marshalledResponseBody,
683 }
684
685}
686
687//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
688//or an error on failure
689func encodeResponse(request *ic.InterContainerMessage, success bool, returnedValues ...interface{}) (*ic.InterContainerMessage, error) {
Esin Karamanccb714b2019-11-29 15:02:06 +0000690 //logger.Debugw("encodeResponse", log.Fields{"success": success, "returnedValues": returnedValues})
William Kurkianea869482019-04-09 15:16:11 -0400691 responseHeader := &ic.Header{
692 Id: request.Header.Id,
693 Type: ic.MessageType_RESPONSE,
694 FromTopic: request.Header.ToTopic,
695 ToTopic: request.Header.FromTopic,
Matt Jeanneret384d8c92019-05-06 14:27:31 -0400696 KeyTopic: request.Header.KeyTopic,
Scott Bakered4a8e72020-04-17 11:10:20 -0700697 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400698 }
699
700 // Go over all returned values
701 var marshalledReturnedVal *any.Any
702 var err error
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000703
704 // for now we support only 1 returned value - (excluding the error)
705 if len(returnedValues) > 0 {
706 if marshalledReturnedVal, err = encodeReturnedValue(returnedValues[0]); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000707 logger.Warnw("cannot-marshal-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400708 }
William Kurkianea869482019-04-09 15:16:11 -0400709 }
710
711 responseBody := &ic.InterContainerResponseBody{
712 Success: success,
713 Result: marshalledReturnedVal,
714 }
715
716 // Marshal the response body
717 var marshalledResponseBody *any.Any
718 if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000719 logger.Warnw("cannot-marshal-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400720 return nil, err
721 }
722
723 return &ic.InterContainerMessage{
724 Header: responseHeader,
725 Body: marshalledResponseBody,
726 }, nil
727}
728
729func CallFuncByName(myClass interface{}, funcName string, params ...interface{}) (out []reflect.Value, err error) {
730 myClassValue := reflect.ValueOf(myClass)
731 // Capitalize the first letter in the funcName to workaround the first capital letters required to
732 // invoke a function from a different package
733 funcName = strings.Title(funcName)
734 m := myClassValue.MethodByName(funcName)
735 if !m.IsValid() {
736 return make([]reflect.Value, 0), fmt.Errorf("method-not-found \"%s\"", funcName)
737 }
738 in := make([]reflect.Value, len(params))
739 for i, param := range params {
740 in[i] = reflect.ValueOf(param)
741 }
742 out = m.Call(in)
743 return
744}
745
npujarec5762e2020-01-01 14:08:48 +0530746func (kp *interContainerProxy) addTransactionId(transactionId string, currentArgs []*ic.Argument) []*ic.Argument {
William Kurkianea869482019-04-09 15:16:11 -0400747 arg := &KVArg{
748 Key: TransactionKey,
749 Value: &ic.StrType{Val: transactionId},
750 }
751
752 var marshalledArg *any.Any
753 var err error
754 if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: transactionId}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000755 logger.Warnw("cannot-add-transactionId", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400756 return currentArgs
757 }
758 protoArg := &ic.Argument{
759 Key: arg.Key,
760 Value: marshalledArg,
761 }
762 return append(currentArgs, protoArg)
763}
764
npujarec5762e2020-01-01 14:08:48 +0530765func (kp *interContainerProxy) addFromTopic(fromTopic string, currentArgs []*ic.Argument) []*ic.Argument {
William Kurkianea869482019-04-09 15:16:11 -0400766 var marshalledArg *any.Any
767 var err error
768 if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: fromTopic}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000769 logger.Warnw("cannot-add-transactionId", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400770 return currentArgs
771 }
772 protoArg := &ic.Argument{
773 Key: FromTopic,
774 Value: marshalledArg,
775 }
776 return append(currentArgs, protoArg)
777}
778
npujarec5762e2020-01-01 14:08:48 +0530779func (kp *interContainerProxy) handleMessage(msg *ic.InterContainerMessage, targetInterface interface{}) {
William Kurkianea869482019-04-09 15:16:11 -0400780
781 // First extract the header to know whether this is a request - responses are handled by a different handler
782 if msg.Header.Type == ic.MessageType_REQUEST {
783 var out []reflect.Value
784 var err error
785
786 // Get the request body
787 requestBody := &ic.InterContainerRequestBody{}
788 if err = ptypes.UnmarshalAny(msg.Body, requestBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000789 logger.Warnw("cannot-unmarshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400790 } else {
Esin Karamanccb714b2019-11-29 15:02:06 +0000791 logger.Debugw("received-request", log.Fields{"rpc": requestBody.Rpc, "header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400792 // let the callee unpack the arguments as its the only one that knows the real proto type
793 // Augment the requestBody with the message Id as it will be used in scenarios where cores
794 // are set in pairs and competing
795 requestBody.Args = kp.addTransactionId(msg.Header.Id, requestBody.Args)
796
797 // Augment the requestBody with the From topic name as it will be used in scenarios where a container
798 // needs to send an unsollicited message to the currently requested container
799 requestBody.Args = kp.addFromTopic(msg.Header.FromTopic, requestBody.Args)
800
801 out, err = CallFuncByName(targetInterface, requestBody.Rpc, requestBody.Args)
802 if err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000803 logger.Warn(err)
William Kurkianea869482019-04-09 15:16:11 -0400804 }
805 }
806 // Response required?
807 if requestBody.ResponseRequired {
808 // If we already have an error before then just return that
809 var returnError *ic.Error
810 var returnedValues []interface{}
811 var success bool
812 if err != nil {
813 returnError = &ic.Error{Reason: err.Error()}
814 returnedValues = make([]interface{}, 1)
815 returnedValues[0] = returnError
816 } else {
817 returnedValues = make([]interface{}, 0)
818 // Check for errors first
819 lastIndex := len(out) - 1
820 if out[lastIndex].Interface() != nil { // Error
kdarapub26b4502019-10-05 03:02:33 +0530821 if retError, ok := out[lastIndex].Interface().(error); ok {
822 if retError.Error() == ErrorTransactionNotAcquired.Error() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000823 logger.Debugw("Ignoring request", log.Fields{"error": retError, "txId": msg.Header.Id})
kdarapub26b4502019-10-05 03:02:33 +0530824 return // Ignore - process is in competing mode and ignored transaction
825 }
826 returnError = &ic.Error{Reason: retError.Error()}
William Kurkianea869482019-04-09 15:16:11 -0400827 returnedValues = append(returnedValues, returnError)
828 } else { // Should never happen
829 returnError = &ic.Error{Reason: "incorrect-error-returns"}
830 returnedValues = append(returnedValues, returnError)
831 }
832 } else if len(out) == 2 && reflect.ValueOf(out[0].Interface()).IsValid() && reflect.ValueOf(out[0].Interface()).IsNil() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000833 logger.Warnw("Unexpected response of (nil,nil)", log.Fields{"txId": msg.Header.Id})
kdarapub26b4502019-10-05 03:02:33 +0530834 return // Ignore - should not happen
William Kurkianea869482019-04-09 15:16:11 -0400835 } else { // Non-error case
836 success = true
837 for idx, val := range out {
Esin Karamanccb714b2019-11-29 15:02:06 +0000838 //logger.Debugw("returned-api-response-loop", log.Fields{"idx": idx, "val": val.Interface()})
William Kurkianea869482019-04-09 15:16:11 -0400839 if idx != lastIndex {
840 returnedValues = append(returnedValues, val.Interface())
841 }
842 }
843 }
844 }
845
846 var icm *ic.InterContainerMessage
847 if icm, err = encodeResponse(msg, success, returnedValues...); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000848 logger.Warnw("error-encoding-response-returning-failure-result", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400849 icm = encodeDefaultFailedResponse(msg)
850 }
851 // To preserve ordering of messages, all messages to a given topic are sent to the same partition
852 // by providing a message key. The key is encoded in the topic name. If the deviceId is not
853 // present then the key will be empty, hence all messages for a given topic will be sent to all
854 // partitions.
855 replyTopic := &Topic{Name: msg.Header.FromTopic}
856 key := msg.Header.KeyTopic
Esin Karamanccb714b2019-11-29 15:02:06 +0000857 logger.Debugw("sending-response-to-kafka", log.Fields{"rpc": requestBody.Rpc, "header": icm.Header, "key": key})
William Kurkianea869482019-04-09 15:16:11 -0400858 // TODO: handle error response.
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000859 go func() {
860 if err := kp.kafkaClient.Send(icm, replyTopic, key); err != nil {
861 logger.Errorw("send-reply-failed", log.Fields{
862 "topic": replyTopic,
863 "key": key,
864 "error": err})
865 }
866 }()
William Kurkianea869482019-04-09 15:16:11 -0400867 }
868 } else if msg.Header.Type == ic.MessageType_RESPONSE {
Esin Karamanccb714b2019-11-29 15:02:06 +0000869 logger.Debugw("response-received", log.Fields{"msg-header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400870 go kp.dispatchResponse(msg)
871 } else {
Esin Karamanccb714b2019-11-29 15:02:06 +0000872 logger.Warnw("unsupported-message-received", log.Fields{"msg-header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400873 }
874}
875
npujarec5762e2020-01-01 14:08:48 +0530876func (kp *interContainerProxy) waitForMessages(ch <-chan *ic.InterContainerMessage, topic Topic, targetInterface interface{}) {
William Kurkianea869482019-04-09 15:16:11 -0400877 // Wait for messages
878 for msg := range ch {
Esin Karamanccb714b2019-11-29 15:02:06 +0000879 //logger.Debugw("request-received", log.Fields{"msg": msg, "topic": topic.Name, "target": targetInterface})
William Kurkianea869482019-04-09 15:16:11 -0400880 go kp.handleMessage(msg, targetInterface)
881 }
882}
883
npujarec5762e2020-01-01 14:08:48 +0530884func (kp *interContainerProxy) dispatchResponse(msg *ic.InterContainerMessage) {
William Kurkianea869482019-04-09 15:16:11 -0400885 kp.lockTransactionIdToChannelMap.RLock()
886 defer kp.lockTransactionIdToChannelMap.RUnlock()
887 if _, exist := kp.transactionIdToChannelMap[msg.Header.Id]; !exist {
Esin Karamanccb714b2019-11-29 15:02:06 +0000888 logger.Debugw("no-waiting-channel", log.Fields{"transaction": msg.Header.Id})
William Kurkianea869482019-04-09 15:16:11 -0400889 return
890 }
891 kp.transactionIdToChannelMap[msg.Header.Id].ch <- msg
892}
893
894// subscribeForResponse allows a caller to subscribe to a given topic when waiting for a response.
895// This method is built to prevent all subscribers to receive all messages as is the case of the Subscribe
896// API. There is one response channel waiting for kafka messages before dispatching the message to the
897// corresponding waiting channel
npujarec5762e2020-01-01 14:08:48 +0530898func (kp *interContainerProxy) subscribeForResponse(topic Topic, trnsId string) (chan *ic.InterContainerMessage, error) {
Esin Karamanccb714b2019-11-29 15:02:06 +0000899 logger.Debugw("subscribeForResponse", log.Fields{"topic": topic.Name, "trnsid": trnsId})
William Kurkianea869482019-04-09 15:16:11 -0400900
901 // Create a specific channel for this consumers. We cannot use the channel from the kafkaclient as it will
902 // broadcast any message for this topic to all channels waiting on it.
divyadesaid26f6b12020-03-19 06:30:28 +0000903 // Set channel size to 1 to prevent deadlock, see VOL-2708
904 ch := make(chan *ic.InterContainerMessage, 1)
William Kurkianea869482019-04-09 15:16:11 -0400905 kp.addToTransactionIdToChannelMap(trnsId, &topic, ch)
906
907 return ch, nil
908}
909
npujarec5762e2020-01-01 14:08:48 +0530910func (kp *interContainerProxy) unSubscribeForResponse(trnsId string) error {
Esin Karamanccb714b2019-11-29 15:02:06 +0000911 logger.Debugw("unsubscribe-for-response", log.Fields{"trnsId": trnsId})
William Kurkianea869482019-04-09 15:16:11 -0400912 kp.deleteFromTransactionIdToChannelMap(trnsId)
913 return nil
914}
915
npujarec5762e2020-01-01 14:08:48 +0530916func (kp *interContainerProxy) EnableLivenessChannel(enable bool) chan bool {
cbabu95f21522019-11-13 14:25:18 +0100917 return kp.kafkaClient.EnableLivenessChannel(enable)
918}
919
npujarec5762e2020-01-01 14:08:48 +0530920func (kp *interContainerProxy) EnableHealthinessChannel(enable bool) chan bool {
Scott Baker86fce9a2019-12-12 09:47:17 -0800921 return kp.kafkaClient.EnableHealthinessChannel(enable)
922}
923
npujarec5762e2020-01-01 14:08:48 +0530924func (kp *interContainerProxy) SendLiveness() error {
cbabu95f21522019-11-13 14:25:18 +0100925 return kp.kafkaClient.SendLiveness()
926}
927
William Kurkianea869482019-04-09 15:16:11 -0400928//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
929//or an error on failure
930func encodeRequest(rpc string, toTopic *Topic, replyTopic *Topic, key string, kvArgs ...*KVArg) (*ic.InterContainerMessage, error) {
931 requestHeader := &ic.Header{
932 Id: uuid.New().String(),
933 Type: ic.MessageType_REQUEST,
934 FromTopic: replyTopic.Name,
935 ToTopic: toTopic.Name,
Matt Jeanneret384d8c92019-05-06 14:27:31 -0400936 KeyTopic: key,
Scott Bakered4a8e72020-04-17 11:10:20 -0700937 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400938 }
939 requestBody := &ic.InterContainerRequestBody{
940 Rpc: rpc,
941 ResponseRequired: true,
942 ReplyToTopic: replyTopic.Name,
943 }
944
945 for _, arg := range kvArgs {
946 if arg == nil {
947 // In case the caller sends an array with empty args
948 continue
949 }
950 var marshalledArg *any.Any
951 var err error
952 // ascertain the value interface type is a proto.Message
953 protoValue, ok := arg.Value.(proto.Message)
954 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000955 logger.Warnw("argument-value-not-proto-message", log.Fields{"error": ok, "Value": arg.Value})
William Kurkianea869482019-04-09 15:16:11 -0400956 err := errors.New("argument-value-not-proto-message")
957 return nil, err
958 }
959 if marshalledArg, err = ptypes.MarshalAny(protoValue); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000960 logger.Warnw("cannot-marshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400961 return nil, err
962 }
963 protoArg := &ic.Argument{
964 Key: arg.Key,
965 Value: marshalledArg,
966 }
967 requestBody.Args = append(requestBody.Args, protoArg)
968 }
969
970 var marshalledData *any.Any
971 var err error
972 if marshalledData, err = ptypes.MarshalAny(requestBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000973 logger.Warnw("cannot-marshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400974 return nil, err
975 }
976 request := &ic.InterContainerMessage{
977 Header: requestHeader,
978 Body: marshalledData,
979 }
980 return request, nil
981}
982
983func decodeResponse(response *ic.InterContainerMessage) (*ic.InterContainerResponseBody, error) {
984 // Extract the message body
985 responseBody := ic.InterContainerResponseBody{}
986 if err := ptypes.UnmarshalAny(response.Body, &responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000987 logger.Warnw("cannot-unmarshal-response", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400988 return nil, err
989 }
Esin Karamanccb714b2019-11-29 15:02:06 +0000990 //logger.Debugw("response-decoded-successfully", log.Fields{"response-status": &responseBody.Success})
William Kurkianea869482019-04-09 15:16:11 -0400991
992 return &responseBody, nil
993
994}