blob: cbde83421c893cf5980f8eb9731b9f2e103e7284 [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"
William Kurkianea869482019-04-09 15:16:11 -040022 "reflect"
23 "strings"
24 "sync"
25 "time"
William Kurkianea869482019-04-09 15:16:11 -040026
Andrea Campanella974d7452020-06-26 19:32:30 +020027 "google.golang.org/grpc/codes"
28 "google.golang.org/grpc/status"
29
Esin Karamanccb714b2019-11-29 15:02:06 +000030 "github.com/golang/protobuf/proto"
31 "github.com/golang/protobuf/ptypes"
32 "github.com/golang/protobuf/ptypes/any"
33 "github.com/google/uuid"
34 "github.com/opencord/voltha-lib-go/v3/pkg/log"
35 ic "github.com/opencord/voltha-protos/v3/go/inter_container"
36)
William Kurkianea869482019-04-09 15:16:11 -040037
38const (
39 DefaultMaxRetries = 3
divyadesaid26f6b12020-03-19 06:30:28 +000040 DefaultRequestTimeout = 60000 // 60000 milliseconds - to handle a wider latency range
William Kurkianea869482019-04-09 15:16:11 -040041)
42
43const (
44 TransactionKey = "transactionID"
45 FromTopic = "fromTopic"
46)
47
kdarapub26b4502019-10-05 03:02:33 +053048var ErrorTransactionNotAcquired = errors.New("transaction-not-acquired")
49var ErrorTransactionInvalidId = errors.New("transaction-invalid-id")
50
William Kurkianea869482019-04-09 15:16:11 -040051// requestHandlerChannel represents an interface associated with a channel. Whenever, an event is
52// obtained from that channel, this interface is invoked. This is used to handle
53// async requests into the Core via the kafka messaging bus
54type requestHandlerChannel struct {
55 requesthandlerInterface interface{}
56 ch <-chan *ic.InterContainerMessage
57}
58
59// transactionChannel represents a combination of a topic and a channel onto which a response received
60// on the kafka bus will be sent to
61type transactionChannel struct {
62 topic *Topic
63 ch chan *ic.InterContainerMessage
64}
65
npujarec5762e2020-01-01 14:08:48 +053066type InterContainerProxy interface {
67 Start() error
68 Stop()
69 GetDefaultTopic() *Topic
npujarec5762e2020-01-01 14:08:48 +053070 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 {
Neha Sharma3f221ae2020-04-29 19:02:12 +000082 kafkaAddress string
npujarec5762e2020-01-01 14:08:48 +053083 defaultTopic *Topic
William Kurkianea869482019-04-09 15:16:11 -040084 defaultRequestHandlerInterface interface{}
William Kurkianea869482019-04-09 15:16:11 -040085 kafkaClient Client
npujarec5762e2020-01-01 14:08:48 +053086 doneCh chan struct{}
87 doneOnce sync.Once
William Kurkianea869482019-04-09 15:16:11 -040088
89 // This map is used to map a topic to an interface and channel. When a request is received
90 // on that channel (registered to the topic) then that interface is invoked.
91 topicToRequestHandlerChannelMap map[string]*requestHandlerChannel
92 lockTopicRequestHandlerChannelMap sync.RWMutex
93
94 // This map is used to map a channel to a response topic. This channel handles all responses on that
95 // channel for that topic and forward them to the appropriate consumers channel, using the
96 // transactionIdToChannelMap.
97 topicToResponseChannelMap map[string]<-chan *ic.InterContainerMessage
98 lockTopicResponseChannelMap sync.RWMutex
99
100 // This map is used to map a transaction to a consumers channel. This is used whenever a request has been
101 // sent out and we are waiting for a response.
102 transactionIdToChannelMap map[string]*transactionChannel
103 lockTransactionIdToChannelMap sync.RWMutex
104}
105
npujarec5762e2020-01-01 14:08:48 +0530106type InterContainerProxyOption func(*interContainerProxy)
William Kurkianea869482019-04-09 15:16:11 -0400107
Neha Sharma3f221ae2020-04-29 19:02:12 +0000108func InterContainerAddress(address string) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530109 return func(args *interContainerProxy) {
Neha Sharma3f221ae2020-04-29 19:02:12 +0000110 args.kafkaAddress = address
William Kurkianea869482019-04-09 15:16:11 -0400111 }
112}
113
114func DefaultTopic(topic *Topic) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530115 return func(args *interContainerProxy) {
116 args.defaultTopic = topic
William Kurkianea869482019-04-09 15:16:11 -0400117 }
118}
119
William Kurkianea869482019-04-09 15:16:11 -0400120func RequestHandlerInterface(handler interface{}) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530121 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400122 args.defaultRequestHandlerInterface = handler
123 }
124}
125
126func MsgClient(client Client) InterContainerProxyOption {
npujarec5762e2020-01-01 14:08:48 +0530127 return func(args *interContainerProxy) {
William Kurkianea869482019-04-09 15:16:11 -0400128 args.kafkaClient = client
129 }
130}
131
npujarec5762e2020-01-01 14:08:48 +0530132func newInterContainerProxy(opts ...InterContainerProxyOption) *interContainerProxy {
133 proxy := &interContainerProxy{
Neha Sharma3f221ae2020-04-29 19:02:12 +0000134 kafkaAddress: DefaultKafkaAddress,
135 doneCh: make(chan struct{}),
William Kurkianea869482019-04-09 15:16:11 -0400136 }
137
138 for _, option := range opts {
139 option(proxy)
140 }
141
npujarec5762e2020-01-01 14:08:48 +0530142 return proxy
William Kurkianea869482019-04-09 15:16:11 -0400143}
144
npujarec5762e2020-01-01 14:08:48 +0530145func NewInterContainerProxy(opts ...InterContainerProxyOption) InterContainerProxy {
146 return newInterContainerProxy(opts...)
147}
148
149func (kp *interContainerProxy) Start() error {
Esin Karamanccb714b2019-11-29 15:02:06 +0000150 logger.Info("Starting-Proxy")
William Kurkianea869482019-04-09 15:16:11 -0400151
152 // Kafka MsgClient should already have been created. If not, output fatal error
153 if kp.kafkaClient == nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000154 logger.Fatal("kafka-client-not-set")
William Kurkianea869482019-04-09 15:16:11 -0400155 }
156
William Kurkianea869482019-04-09 15:16:11 -0400157 // Start the kafka client
158 if err := kp.kafkaClient.Start(); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000159 logger.Errorw("Cannot-create-kafka-proxy", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400160 return err
161 }
162
163 // Create the topic to response channel map
164 kp.topicToResponseChannelMap = make(map[string]<-chan *ic.InterContainerMessage)
165 //
166 // Create the transactionId to Channel Map
167 kp.transactionIdToChannelMap = make(map[string]*transactionChannel)
168
169 // Create the topic to request channel map
170 kp.topicToRequestHandlerChannelMap = make(map[string]*requestHandlerChannel)
171
172 return nil
173}
174
npujarec5762e2020-01-01 14:08:48 +0530175func (kp *interContainerProxy) Stop() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000176 logger.Info("stopping-intercontainer-proxy")
npujarec5762e2020-01-01 14:08:48 +0530177 kp.doneOnce.Do(func() { close(kp.doneCh) })
William Kurkianea869482019-04-09 15:16:11 -0400178 // TODO : Perform cleanup
179 kp.kafkaClient.Stop()
Scott Bakere701b862020-02-20 16:19:16 -0800180 err := kp.deleteAllTopicRequestHandlerChannelMap()
181 if err != nil {
Scott Baker24f83e22020-03-30 16:14:28 -0700182 logger.Errorw("failed-delete-all-topic-request-handler-channel-map", log.Fields{"error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800183 }
184 err = kp.deleteAllTopicResponseChannelMap()
185 if err != nil {
Scott Baker24f83e22020-03-30 16:14:28 -0700186 logger.Errorw("failed-delete-all-topic-response-channel-map", log.Fields{"error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800187 }
188 kp.deleteAllTransactionIdToChannelMap()
William Kurkianea869482019-04-09 15:16:11 -0400189}
190
npujarec5762e2020-01-01 14:08:48 +0530191func (kp *interContainerProxy) GetDefaultTopic() *Topic {
192 return kp.defaultTopic
193}
194
divyadesaid26f6b12020-03-19 06:30:28 +0000195// InvokeAsyncRPC is used to make an RPC request asynchronously
196func (kp *interContainerProxy) InvokeAsyncRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
197 waitForResponse bool, key string, kvArgs ...*KVArg) chan *RpcResponse {
198
199 logger.Debugw("InvokeAsyncRPC", log.Fields{"rpc": rpc, "key": key})
200 // If a replyToTopic is provided then we use it, otherwise just use the default toTopic. The replyToTopic is
201 // typically the device ID.
202 responseTopic := replyToTopic
203 if responseTopic == nil {
204 responseTopic = kp.GetDefaultTopic()
205 }
206
207 chnl := make(chan *RpcResponse)
208
209 go func() {
210
211 // once we're done,
212 // close the response channel
213 defer close(chnl)
214
215 var err error
216 var protoRequest *ic.InterContainerMessage
217
218 // Encode the request
219 protoRequest, err = encodeRequest(rpc, toTopic, responseTopic, key, kvArgs...)
220 if err != nil {
221 logger.Warnw("cannot-format-request", log.Fields{"rpc": rpc, "error": err})
222 chnl <- NewResponse(RpcFormattingError, err, nil)
223 return
224 }
225
226 // Subscribe for response, if needed, before sending request
227 var ch <-chan *ic.InterContainerMessage
228 if ch, err = kp.subscribeForResponse(*responseTopic, protoRequest.Header.Id); err != nil {
229 logger.Errorw("failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
230 chnl <- NewResponse(RpcTransportError, err, nil)
231 return
232 }
233
234 // Send request - if the topic is formatted with a device Id then we will send the request using a
235 // specific key, hence ensuring a single partition is used to publish the request. This ensures that the
236 // subscriber on that topic will receive the request in the order it was sent. The key used is the deviceId.
237 logger.Debugw("sending-msg", log.Fields{"rpc": rpc, "toTopic": toTopic, "replyTopic": responseTopic, "key": key, "xId": protoRequest.Header.Id})
238
239 // if the message is not sent on kafka publish an event an close the channel
240 if err = kp.kafkaClient.Send(protoRequest, toTopic, key); err != nil {
241 chnl <- NewResponse(RpcTransportError, err, nil)
242 return
243 }
244
245 // if the client is not waiting for a response send the ack and close the channel
246 chnl <- NewResponse(RpcSent, nil, nil)
247 if !waitForResponse {
248 return
249 }
250
251 defer func() {
252 // Remove the subscription for a response on return
253 if err := kp.unSubscribeForResponse(protoRequest.Header.Id); err != nil {
254 logger.Warnw("invoke-async-rpc-unsubscriber-for-response-failed", log.Fields{"err": err})
255 }
256 }()
257
258 // Wait for response as well as timeout or cancellation
259 select {
260 case msg, ok := <-ch:
261 if !ok {
262 logger.Warnw("channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
263 chnl <- NewResponse(RpcTransportError, status.Error(codes.Aborted, "channel closed"), nil)
264 }
265 logger.Debugw("received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
266 if responseBody, err := decodeResponse(msg); err != nil {
267 chnl <- NewResponse(RpcReply, err, nil)
268 } else {
269 if responseBody.Success {
270 chnl <- NewResponse(RpcReply, nil, responseBody.Result)
271 } else {
272 // response body contains an error
273 unpackErr := &ic.Error{}
274 if err := ptypes.UnmarshalAny(responseBody.Result, unpackErr); err != nil {
275 chnl <- NewResponse(RpcReply, err, nil)
276 } else {
277 chnl <- NewResponse(RpcReply, status.Error(codes.Internal, unpackErr.Reason), nil)
278 }
279 }
280 }
281 case <-ctx.Done():
282 logger.Errorw("context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
283 err := status.Error(codes.DeadlineExceeded, ctx.Err().Error())
284 chnl <- NewResponse(RpcTimeout, err, nil)
285 case <-kp.doneCh:
286 chnl <- NewResponse(RpcSystemClosing, nil, nil)
287 logger.Warnw("received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
288 }
289 }()
290 return chnl
291}
292
William Kurkianea869482019-04-09 15:16:11 -0400293// InvokeRPC is used to send a request to a given topic
npujarec5762e2020-01-01 14:08:48 +0530294func (kp *interContainerProxy) InvokeRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
William Kurkianea869482019-04-09 15:16:11 -0400295 waitForResponse bool, key string, kvArgs ...*KVArg) (bool, *any.Any) {
296
297 // If a replyToTopic is provided then we use it, otherwise just use the default toTopic. The replyToTopic is
298 // typically the device ID.
299 responseTopic := replyToTopic
300 if responseTopic == nil {
npujarec5762e2020-01-01 14:08:48 +0530301 responseTopic = kp.defaultTopic
William Kurkianea869482019-04-09 15:16:11 -0400302 }
303
304 // Encode the request
305 protoRequest, err := encodeRequest(rpc, toTopic, responseTopic, key, kvArgs...)
306 if err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000307 logger.Warnw("cannot-format-request", log.Fields{"rpc": rpc, "error": err})
William Kurkianea869482019-04-09 15:16:11 -0400308 return false, nil
309 }
310
311 // Subscribe for response, if needed, before sending request
312 var ch <-chan *ic.InterContainerMessage
313 if waitForResponse {
314 var err error
315 if ch, err = kp.subscribeForResponse(*responseTopic, protoRequest.Header.Id); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000316 logger.Errorw("failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400317 }
318 }
319
320 // Send request - if the topic is formatted with a device Id then we will send the request using a
321 // specific key, hence ensuring a single partition is used to publish the request. This ensures that the
322 // subscriber on that topic will receive the request in the order it was sent. The key used is the deviceId.
323 //key := GetDeviceIdFromTopic(*toTopic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000324 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 +0000325 go func() {
326 if err := kp.kafkaClient.Send(protoRequest, toTopic, key); err != nil {
327 logger.Errorw("send-failed", log.Fields{
328 "topic": toTopic,
329 "key": key,
330 "error": err})
331 }
332 }()
William Kurkianea869482019-04-09 15:16:11 -0400333
334 if waitForResponse {
335 // Create a child context based on the parent context, if any
336 var cancel context.CancelFunc
337 childCtx := context.Background()
338 if ctx == nil {
339 ctx, cancel = context.WithTimeout(context.Background(), DefaultRequestTimeout*time.Millisecond)
340 } else {
341 childCtx, cancel = context.WithTimeout(ctx, DefaultRequestTimeout*time.Millisecond)
342 }
343 defer cancel()
344
345 // Wait for response as well as timeout or cancellation
346 // Remove the subscription for a response on return
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000347 defer func() {
348 if err := kp.unSubscribeForResponse(protoRequest.Header.Id); err != nil {
349 logger.Errorw("response-unsubscribe-failed", log.Fields{
350 "id": protoRequest.Header.Id,
351 "error": err})
352 }
353 }()
William Kurkianea869482019-04-09 15:16:11 -0400354 select {
355 case msg, ok := <-ch:
356 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000357 logger.Warnw("channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400358 protoError := &ic.Error{Reason: "channel-closed"}
359 var marshalledArg *any.Any
360 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
361 return false, nil // Should never happen
362 }
363 return false, marshalledArg
364 }
Esin Karamanccb714b2019-11-29 15:02:06 +0000365 logger.Debugw("received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400366 var responseBody *ic.InterContainerResponseBody
367 var err error
368 if responseBody, err = decodeResponse(msg); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000369 logger.Errorw("decode-response-error", log.Fields{"error": err})
npujarec5762e2020-01-01 14:08:48 +0530370 // FIXME we should return something
William Kurkianea869482019-04-09 15:16:11 -0400371 }
372 return responseBody.Success, responseBody.Result
373 case <-ctx.Done():
Esin Karamanccb714b2019-11-29 15:02:06 +0000374 logger.Debugw("context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
William Kurkianea869482019-04-09 15:16:11 -0400375 // pack the error as proto any type
npujarec5762e2020-01-01 14:08:48 +0530376 protoError := &ic.Error{Reason: ctx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
377
William Kurkianea869482019-04-09 15:16:11 -0400378 var marshalledArg *any.Any
379 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
380 return false, nil // Should never happen
381 }
382 return false, marshalledArg
383 case <-childCtx.Done():
Esin Karamanccb714b2019-11-29 15:02:06 +0000384 logger.Debugw("context-cancelled", log.Fields{"rpc": rpc, "ctx": childCtx.Err()})
William Kurkianea869482019-04-09 15:16:11 -0400385 // pack the error as proto any type
npujarec5762e2020-01-01 14:08:48 +0530386 protoError := &ic.Error{Reason: childCtx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
387
William Kurkianea869482019-04-09 15:16:11 -0400388 var marshalledArg *any.Any
389 if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
390 return false, nil // Should never happen
391 }
392 return false, marshalledArg
393 case <-kp.doneCh:
Esin Karamanccb714b2019-11-29 15:02:06 +0000394 logger.Infow("received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
William Kurkianea869482019-04-09 15:16:11 -0400395 return true, nil
396 }
397 }
398 return true, nil
399}
400
401// SubscribeWithRequestHandlerInterface allows a caller to assign a target object to be invoked automatically
402// when a message is received on a given topic
npujarec5762e2020-01-01 14:08:48 +0530403func (kp *interContainerProxy) SubscribeWithRequestHandlerInterface(topic Topic, handler interface{}) error {
William Kurkianea869482019-04-09 15:16:11 -0400404
405 // Subscribe to receive messages for that topic
406 var ch <-chan *ic.InterContainerMessage
407 var err error
408 if ch, err = kp.kafkaClient.Subscribe(&topic); err != nil {
409 //if ch, err = kp.Subscribe(topic); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000410 logger.Errorw("failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
Matt Jeannereteb5059f2019-07-19 06:11:00 -0400411 return err
William Kurkianea869482019-04-09 15:16:11 -0400412 }
413
414 kp.defaultRequestHandlerInterface = handler
415 kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: handler, ch: ch})
416 // Launch a go routine to receive and process kafka messages
417 go kp.waitForMessages(ch, topic, handler)
418
419 return nil
420}
421
422// SubscribeWithDefaultRequestHandler allows a caller to add a topic to an existing target object to be invoked automatically
423// 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 +0530424func (kp *interContainerProxy) SubscribeWithDefaultRequestHandler(topic Topic, initialOffset int64) error {
William Kurkianea869482019-04-09 15:16:11 -0400425 // Subscribe to receive messages for that topic
426 var ch <-chan *ic.InterContainerMessage
427 var err error
428 if ch, err = kp.kafkaClient.Subscribe(&topic, &KVArg{Key: Offset, Value: initialOffset}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000429 logger.Errorw("failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
William Kurkianea869482019-04-09 15:16:11 -0400430 return err
431 }
432 kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: kp.defaultRequestHandlerInterface, ch: ch})
433
434 // Launch a go routine to receive and process kafka messages
435 go kp.waitForMessages(ch, topic, kp.defaultRequestHandlerInterface)
436
437 return nil
438}
439
npujarec5762e2020-01-01 14:08:48 +0530440func (kp *interContainerProxy) UnSubscribeFromRequestHandler(topic Topic) error {
William Kurkianea869482019-04-09 15:16:11 -0400441 return kp.deleteFromTopicRequestHandlerChannelMap(topic.Name)
442}
443
npujarec5762e2020-01-01 14:08:48 +0530444func (kp *interContainerProxy) deleteFromTopicResponseChannelMap(topic string) error {
William Kurkianea869482019-04-09 15:16:11 -0400445 kp.lockTopicResponseChannelMap.Lock()
446 defer kp.lockTopicResponseChannelMap.Unlock()
447 if _, exist := kp.topicToResponseChannelMap[topic]; exist {
448 // Unsubscribe to this topic first - this will close the subscribed channel
449 var err error
450 if err = kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000451 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic})
William Kurkianea869482019-04-09 15:16:11 -0400452 }
453 delete(kp.topicToResponseChannelMap, topic)
454 return err
455 } else {
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000456 return fmt.Errorf("%s-Topic-not-found", topic)
William Kurkianea869482019-04-09 15:16:11 -0400457 }
458}
459
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000460// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530461func (kp *interContainerProxy) deleteAllTopicResponseChannelMap() error {
Scott Bakere701b862020-02-20 16:19:16 -0800462 logger.Debug("delete-all-topic-response-channel")
William Kurkianea869482019-04-09 15:16:11 -0400463 kp.lockTopicResponseChannelMap.Lock()
464 defer kp.lockTopicResponseChannelMap.Unlock()
Scott Bakere701b862020-02-20 16:19:16 -0800465 var unsubscribeFailTopics []string
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000466 for topic := range kp.topicToResponseChannelMap {
William Kurkianea869482019-04-09 15:16:11 -0400467 // Unsubscribe to this topic first - this will close the subscribed channel
Scott Bakere701b862020-02-20 16:19:16 -0800468 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
469 unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000470 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic, "error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800471 // Do not return. Continue to try to unsubscribe to other topics.
472 } else {
473 // Only delete from channel map if successfully unsubscribed.
474 delete(kp.topicToResponseChannelMap, topic)
William Kurkianea869482019-04-09 15:16:11 -0400475 }
William Kurkianea869482019-04-09 15:16:11 -0400476 }
Scott Bakere701b862020-02-20 16:19:16 -0800477 if len(unsubscribeFailTopics) > 0 {
478 return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
479 }
480 return nil
William Kurkianea869482019-04-09 15:16:11 -0400481}
482
npujarec5762e2020-01-01 14:08:48 +0530483func (kp *interContainerProxy) addToTopicRequestHandlerChannelMap(topic string, arg *requestHandlerChannel) {
William Kurkianea869482019-04-09 15:16:11 -0400484 kp.lockTopicRequestHandlerChannelMap.Lock()
485 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
486 if _, exist := kp.topicToRequestHandlerChannelMap[topic]; !exist {
487 kp.topicToRequestHandlerChannelMap[topic] = arg
488 }
489}
490
npujarec5762e2020-01-01 14:08:48 +0530491func (kp *interContainerProxy) deleteFromTopicRequestHandlerChannelMap(topic string) error {
William Kurkianea869482019-04-09 15:16:11 -0400492 kp.lockTopicRequestHandlerChannelMap.Lock()
493 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
494 if _, exist := kp.topicToRequestHandlerChannelMap[topic]; exist {
495 // Close the kafka client client first by unsubscribing to this topic
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000496 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
497 return err
498 }
William Kurkianea869482019-04-09 15:16:11 -0400499 delete(kp.topicToRequestHandlerChannelMap, topic)
500 return nil
501 } else {
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000502 return fmt.Errorf("%s-Topic-not-found", topic)
William Kurkianea869482019-04-09 15:16:11 -0400503 }
504}
505
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000506// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530507func (kp *interContainerProxy) deleteAllTopicRequestHandlerChannelMap() error {
Scott Bakere701b862020-02-20 16:19:16 -0800508 logger.Debug("delete-all-topic-request-channel")
William Kurkianea869482019-04-09 15:16:11 -0400509 kp.lockTopicRequestHandlerChannelMap.Lock()
510 defer kp.lockTopicRequestHandlerChannelMap.Unlock()
Scott Bakere701b862020-02-20 16:19:16 -0800511 var unsubscribeFailTopics []string
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000512 for topic := range kp.topicToRequestHandlerChannelMap {
William Kurkianea869482019-04-09 15:16:11 -0400513 // Close the kafka client client first by unsubscribing to this topic
Scott Bakere701b862020-02-20 16:19:16 -0800514 if err := kp.kafkaClient.UnSubscribe(&Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
515 unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
Esin Karamanccb714b2019-11-29 15:02:06 +0000516 logger.Errorw("unsubscribing-error", log.Fields{"topic": topic, "error": err})
Scott Bakere701b862020-02-20 16:19:16 -0800517 // Do not return. Continue to try to unsubscribe to other topics.
518 } else {
519 // Only delete from channel map if successfully unsubscribed.
520 delete(kp.topicToRequestHandlerChannelMap, topic)
William Kurkianea869482019-04-09 15:16:11 -0400521 }
William Kurkianea869482019-04-09 15:16:11 -0400522 }
Scott Bakere701b862020-02-20 16:19:16 -0800523 if len(unsubscribeFailTopics) > 0 {
524 return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
525 }
526 return nil
William Kurkianea869482019-04-09 15:16:11 -0400527}
528
npujarec5762e2020-01-01 14:08:48 +0530529func (kp *interContainerProxy) addToTransactionIdToChannelMap(id string, topic *Topic, arg chan *ic.InterContainerMessage) {
William Kurkianea869482019-04-09 15:16:11 -0400530 kp.lockTransactionIdToChannelMap.Lock()
531 defer kp.lockTransactionIdToChannelMap.Unlock()
532 if _, exist := kp.transactionIdToChannelMap[id]; !exist {
533 kp.transactionIdToChannelMap[id] = &transactionChannel{topic: topic, ch: arg}
534 }
535}
536
npujarec5762e2020-01-01 14:08:48 +0530537func (kp *interContainerProxy) deleteFromTransactionIdToChannelMap(id string) {
William Kurkianea869482019-04-09 15:16:11 -0400538 kp.lockTransactionIdToChannelMap.Lock()
539 defer kp.lockTransactionIdToChannelMap.Unlock()
540 if transChannel, exist := kp.transactionIdToChannelMap[id]; exist {
541 // Close the channel first
542 close(transChannel.ch)
543 delete(kp.transactionIdToChannelMap, id)
544 }
545}
546
npujarec5762e2020-01-01 14:08:48 +0530547func (kp *interContainerProxy) deleteTopicTransactionIdToChannelMap(id string) {
William Kurkianea869482019-04-09 15:16:11 -0400548 kp.lockTransactionIdToChannelMap.Lock()
549 defer kp.lockTransactionIdToChannelMap.Unlock()
550 for key, value := range kp.transactionIdToChannelMap {
551 if value.topic.Name == id {
552 close(value.ch)
553 delete(kp.transactionIdToChannelMap, key)
554 }
555 }
556}
557
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000558// nolint: unused
npujarec5762e2020-01-01 14:08:48 +0530559func (kp *interContainerProxy) deleteAllTransactionIdToChannelMap() {
Scott Bakere701b862020-02-20 16:19:16 -0800560 logger.Debug("delete-all-transaction-id-channel-map")
William Kurkianea869482019-04-09 15:16:11 -0400561 kp.lockTransactionIdToChannelMap.Lock()
562 defer kp.lockTransactionIdToChannelMap.Unlock()
563 for key, value := range kp.transactionIdToChannelMap {
564 close(value.ch)
565 delete(kp.transactionIdToChannelMap, key)
566 }
567}
568
npujarec5762e2020-01-01 14:08:48 +0530569func (kp *interContainerProxy) DeleteTopic(topic Topic) error {
William Kurkianea869482019-04-09 15:16:11 -0400570 // If we have any consumers on that topic we need to close them
571 if err := kp.deleteFromTopicResponseChannelMap(topic.Name); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000572 logger.Errorw("delete-from-topic-responsechannelmap-failed", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400573 }
574 if err := kp.deleteFromTopicRequestHandlerChannelMap(topic.Name); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000575 logger.Errorw("delete-from-topic-requesthandlerchannelmap-failed", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400576 }
577 kp.deleteTopicTransactionIdToChannelMap(topic.Name)
578
579 return kp.kafkaClient.DeleteTopic(&topic)
580}
581
582func encodeReturnedValue(returnedVal interface{}) (*any.Any, error) {
583 // Encode the response argument - needs to be a proto message
584 if returnedVal == nil {
585 return nil, nil
586 }
587 protoValue, ok := returnedVal.(proto.Message)
588 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000589 logger.Warnw("response-value-not-proto-message", log.Fields{"error": ok, "returnVal": returnedVal})
William Kurkianea869482019-04-09 15:16:11 -0400590 err := errors.New("response-value-not-proto-message")
591 return nil, err
592 }
593
594 // Marshal the returned value, if any
595 var marshalledReturnedVal *any.Any
596 var err error
597 if marshalledReturnedVal, err = ptypes.MarshalAny(protoValue); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000598 logger.Warnw("cannot-marshal-returned-val", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400599 return nil, err
600 }
601 return marshalledReturnedVal, nil
602}
603
604func encodeDefaultFailedResponse(request *ic.InterContainerMessage) *ic.InterContainerMessage {
605 responseHeader := &ic.Header{
606 Id: request.Header.Id,
607 Type: ic.MessageType_RESPONSE,
608 FromTopic: request.Header.ToTopic,
609 ToTopic: request.Header.FromTopic,
Scott Bakered4a8e72020-04-17 11:10:20 -0700610 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400611 }
612 responseBody := &ic.InterContainerResponseBody{
613 Success: false,
614 Result: nil,
615 }
616 var marshalledResponseBody *any.Any
617 var err error
618 // Error should never happen here
619 if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000620 logger.Warnw("cannot-marshal-failed-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400621 }
622
623 return &ic.InterContainerMessage{
624 Header: responseHeader,
625 Body: marshalledResponseBody,
626 }
627
628}
629
630//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
631//or an error on failure
632func encodeResponse(request *ic.InterContainerMessage, success bool, returnedValues ...interface{}) (*ic.InterContainerMessage, error) {
Esin Karamanccb714b2019-11-29 15:02:06 +0000633 //logger.Debugw("encodeResponse", log.Fields{"success": success, "returnedValues": returnedValues})
William Kurkianea869482019-04-09 15:16:11 -0400634 responseHeader := &ic.Header{
635 Id: request.Header.Id,
636 Type: ic.MessageType_RESPONSE,
637 FromTopic: request.Header.ToTopic,
638 ToTopic: request.Header.FromTopic,
Matt Jeanneret384d8c92019-05-06 14:27:31 -0400639 KeyTopic: request.Header.KeyTopic,
Scott Bakered4a8e72020-04-17 11:10:20 -0700640 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400641 }
642
643 // Go over all returned values
644 var marshalledReturnedVal *any.Any
645 var err error
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000646
647 // for now we support only 1 returned value - (excluding the error)
648 if len(returnedValues) > 0 {
649 if marshalledReturnedVal, err = encodeReturnedValue(returnedValues[0]); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000650 logger.Warnw("cannot-marshal-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400651 }
William Kurkianea869482019-04-09 15:16:11 -0400652 }
653
654 responseBody := &ic.InterContainerResponseBody{
655 Success: success,
656 Result: marshalledReturnedVal,
657 }
658
659 // Marshal the response body
660 var marshalledResponseBody *any.Any
661 if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000662 logger.Warnw("cannot-marshal-response-body", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400663 return nil, err
664 }
665
666 return &ic.InterContainerMessage{
667 Header: responseHeader,
668 Body: marshalledResponseBody,
669 }, nil
670}
671
672func CallFuncByName(myClass interface{}, funcName string, params ...interface{}) (out []reflect.Value, err error) {
673 myClassValue := reflect.ValueOf(myClass)
674 // Capitalize the first letter in the funcName to workaround the first capital letters required to
675 // invoke a function from a different package
676 funcName = strings.Title(funcName)
677 m := myClassValue.MethodByName(funcName)
678 if !m.IsValid() {
679 return make([]reflect.Value, 0), fmt.Errorf("method-not-found \"%s\"", funcName)
680 }
681 in := make([]reflect.Value, len(params))
682 for i, param := range params {
683 in[i] = reflect.ValueOf(param)
684 }
685 out = m.Call(in)
686 return
687}
688
npujarec5762e2020-01-01 14:08:48 +0530689func (kp *interContainerProxy) addTransactionId(transactionId string, currentArgs []*ic.Argument) []*ic.Argument {
William Kurkianea869482019-04-09 15:16:11 -0400690 arg := &KVArg{
691 Key: TransactionKey,
692 Value: &ic.StrType{Val: transactionId},
693 }
694
695 var marshalledArg *any.Any
696 var err error
697 if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: transactionId}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000698 logger.Warnw("cannot-add-transactionId", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400699 return currentArgs
700 }
701 protoArg := &ic.Argument{
702 Key: arg.Key,
703 Value: marshalledArg,
704 }
705 return append(currentArgs, protoArg)
706}
707
npujarec5762e2020-01-01 14:08:48 +0530708func (kp *interContainerProxy) addFromTopic(fromTopic string, currentArgs []*ic.Argument) []*ic.Argument {
William Kurkianea869482019-04-09 15:16:11 -0400709 var marshalledArg *any.Any
710 var err error
711 if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: fromTopic}); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000712 logger.Warnw("cannot-add-transactionId", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400713 return currentArgs
714 }
715 protoArg := &ic.Argument{
716 Key: FromTopic,
717 Value: marshalledArg,
718 }
719 return append(currentArgs, protoArg)
720}
721
npujarec5762e2020-01-01 14:08:48 +0530722func (kp *interContainerProxy) handleMessage(msg *ic.InterContainerMessage, targetInterface interface{}) {
William Kurkianea869482019-04-09 15:16:11 -0400723
724 // First extract the header to know whether this is a request - responses are handled by a different handler
725 if msg.Header.Type == ic.MessageType_REQUEST {
726 var out []reflect.Value
727 var err error
728
729 // Get the request body
730 requestBody := &ic.InterContainerRequestBody{}
731 if err = ptypes.UnmarshalAny(msg.Body, requestBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000732 logger.Warnw("cannot-unmarshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400733 } else {
Esin Karamanccb714b2019-11-29 15:02:06 +0000734 logger.Debugw("received-request", log.Fields{"rpc": requestBody.Rpc, "header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400735 // let the callee unpack the arguments as its the only one that knows the real proto type
736 // Augment the requestBody with the message Id as it will be used in scenarios where cores
737 // are set in pairs and competing
738 requestBody.Args = kp.addTransactionId(msg.Header.Id, requestBody.Args)
739
740 // Augment the requestBody with the From topic name as it will be used in scenarios where a container
741 // needs to send an unsollicited message to the currently requested container
742 requestBody.Args = kp.addFromTopic(msg.Header.FromTopic, requestBody.Args)
743
744 out, err = CallFuncByName(targetInterface, requestBody.Rpc, requestBody.Args)
745 if err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000746 logger.Warn(err)
William Kurkianea869482019-04-09 15:16:11 -0400747 }
748 }
749 // Response required?
750 if requestBody.ResponseRequired {
751 // If we already have an error before then just return that
752 var returnError *ic.Error
753 var returnedValues []interface{}
754 var success bool
755 if err != nil {
756 returnError = &ic.Error{Reason: err.Error()}
757 returnedValues = make([]interface{}, 1)
758 returnedValues[0] = returnError
759 } else {
760 returnedValues = make([]interface{}, 0)
761 // Check for errors first
762 lastIndex := len(out) - 1
763 if out[lastIndex].Interface() != nil { // Error
kdarapub26b4502019-10-05 03:02:33 +0530764 if retError, ok := out[lastIndex].Interface().(error); ok {
765 if retError.Error() == ErrorTransactionNotAcquired.Error() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000766 logger.Debugw("Ignoring request", log.Fields{"error": retError, "txId": msg.Header.Id})
kdarapub26b4502019-10-05 03:02:33 +0530767 return // Ignore - process is in competing mode and ignored transaction
768 }
769 returnError = &ic.Error{Reason: retError.Error()}
William Kurkianea869482019-04-09 15:16:11 -0400770 returnedValues = append(returnedValues, returnError)
771 } else { // Should never happen
772 returnError = &ic.Error{Reason: "incorrect-error-returns"}
773 returnedValues = append(returnedValues, returnError)
774 }
775 } else if len(out) == 2 && reflect.ValueOf(out[0].Interface()).IsValid() && reflect.ValueOf(out[0].Interface()).IsNil() {
Esin Karamanccb714b2019-11-29 15:02:06 +0000776 logger.Warnw("Unexpected response of (nil,nil)", log.Fields{"txId": msg.Header.Id})
kdarapub26b4502019-10-05 03:02:33 +0530777 return // Ignore - should not happen
William Kurkianea869482019-04-09 15:16:11 -0400778 } else { // Non-error case
779 success = true
780 for idx, val := range out {
Esin Karamanccb714b2019-11-29 15:02:06 +0000781 //logger.Debugw("returned-api-response-loop", log.Fields{"idx": idx, "val": val.Interface()})
William Kurkianea869482019-04-09 15:16:11 -0400782 if idx != lastIndex {
783 returnedValues = append(returnedValues, val.Interface())
784 }
785 }
786 }
787 }
788
789 var icm *ic.InterContainerMessage
790 if icm, err = encodeResponse(msg, success, returnedValues...); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000791 logger.Warnw("error-encoding-response-returning-failure-result", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400792 icm = encodeDefaultFailedResponse(msg)
793 }
794 // To preserve ordering of messages, all messages to a given topic are sent to the same partition
795 // by providing a message key. The key is encoded in the topic name. If the deviceId is not
796 // present then the key will be empty, hence all messages for a given topic will be sent to all
797 // partitions.
798 replyTopic := &Topic{Name: msg.Header.FromTopic}
799 key := msg.Header.KeyTopic
Esin Karamanccb714b2019-11-29 15:02:06 +0000800 logger.Debugw("sending-response-to-kafka", log.Fields{"rpc": requestBody.Rpc, "header": icm.Header, "key": key})
William Kurkianea869482019-04-09 15:16:11 -0400801 // TODO: handle error response.
Rohan Agrawal02f784d2020-02-14 09:34:02 +0000802 go func() {
803 if err := kp.kafkaClient.Send(icm, replyTopic, key); err != nil {
804 logger.Errorw("send-reply-failed", log.Fields{
805 "topic": replyTopic,
806 "key": key,
807 "error": err})
808 }
809 }()
William Kurkianea869482019-04-09 15:16:11 -0400810 }
811 } else if msg.Header.Type == ic.MessageType_RESPONSE {
Esin Karamanccb714b2019-11-29 15:02:06 +0000812 logger.Debugw("response-received", log.Fields{"msg-header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400813 go kp.dispatchResponse(msg)
814 } else {
Esin Karamanccb714b2019-11-29 15:02:06 +0000815 logger.Warnw("unsupported-message-received", log.Fields{"msg-header": msg.Header})
William Kurkianea869482019-04-09 15:16:11 -0400816 }
817}
818
npujarec5762e2020-01-01 14:08:48 +0530819func (kp *interContainerProxy) waitForMessages(ch <-chan *ic.InterContainerMessage, topic Topic, targetInterface interface{}) {
William Kurkianea869482019-04-09 15:16:11 -0400820 // Wait for messages
821 for msg := range ch {
Esin Karamanccb714b2019-11-29 15:02:06 +0000822 //logger.Debugw("request-received", log.Fields{"msg": msg, "topic": topic.Name, "target": targetInterface})
William Kurkianea869482019-04-09 15:16:11 -0400823 go kp.handleMessage(msg, targetInterface)
824 }
825}
826
npujarec5762e2020-01-01 14:08:48 +0530827func (kp *interContainerProxy) dispatchResponse(msg *ic.InterContainerMessage) {
William Kurkianea869482019-04-09 15:16:11 -0400828 kp.lockTransactionIdToChannelMap.RLock()
829 defer kp.lockTransactionIdToChannelMap.RUnlock()
830 if _, exist := kp.transactionIdToChannelMap[msg.Header.Id]; !exist {
Esin Karamanccb714b2019-11-29 15:02:06 +0000831 logger.Debugw("no-waiting-channel", log.Fields{"transaction": msg.Header.Id})
William Kurkianea869482019-04-09 15:16:11 -0400832 return
833 }
834 kp.transactionIdToChannelMap[msg.Header.Id].ch <- msg
835}
836
837// subscribeForResponse allows a caller to subscribe to a given topic when waiting for a response.
838// This method is built to prevent all subscribers to receive all messages as is the case of the Subscribe
839// API. There is one response channel waiting for kafka messages before dispatching the message to the
840// corresponding waiting channel
npujarec5762e2020-01-01 14:08:48 +0530841func (kp *interContainerProxy) subscribeForResponse(topic Topic, trnsId string) (chan *ic.InterContainerMessage, error) {
Esin Karamanccb714b2019-11-29 15:02:06 +0000842 logger.Debugw("subscribeForResponse", log.Fields{"topic": topic.Name, "trnsid": trnsId})
William Kurkianea869482019-04-09 15:16:11 -0400843
844 // Create a specific channel for this consumers. We cannot use the channel from the kafkaclient as it will
845 // broadcast any message for this topic to all channels waiting on it.
divyadesaid26f6b12020-03-19 06:30:28 +0000846 // Set channel size to 1 to prevent deadlock, see VOL-2708
847 ch := make(chan *ic.InterContainerMessage, 1)
William Kurkianea869482019-04-09 15:16:11 -0400848 kp.addToTransactionIdToChannelMap(trnsId, &topic, ch)
849
850 return ch, nil
851}
852
npujarec5762e2020-01-01 14:08:48 +0530853func (kp *interContainerProxy) unSubscribeForResponse(trnsId string) error {
Esin Karamanccb714b2019-11-29 15:02:06 +0000854 logger.Debugw("unsubscribe-for-response", log.Fields{"trnsId": trnsId})
William Kurkianea869482019-04-09 15:16:11 -0400855 kp.deleteFromTransactionIdToChannelMap(trnsId)
856 return nil
857}
858
npujarec5762e2020-01-01 14:08:48 +0530859func (kp *interContainerProxy) EnableLivenessChannel(enable bool) chan bool {
cbabu95f21522019-11-13 14:25:18 +0100860 return kp.kafkaClient.EnableLivenessChannel(enable)
861}
862
npujarec5762e2020-01-01 14:08:48 +0530863func (kp *interContainerProxy) EnableHealthinessChannel(enable bool) chan bool {
Scott Baker86fce9a2019-12-12 09:47:17 -0800864 return kp.kafkaClient.EnableHealthinessChannel(enable)
865}
866
npujarec5762e2020-01-01 14:08:48 +0530867func (kp *interContainerProxy) SendLiveness() error {
cbabu95f21522019-11-13 14:25:18 +0100868 return kp.kafkaClient.SendLiveness()
869}
870
William Kurkianea869482019-04-09 15:16:11 -0400871//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
872//or an error on failure
873func encodeRequest(rpc string, toTopic *Topic, replyTopic *Topic, key string, kvArgs ...*KVArg) (*ic.InterContainerMessage, error) {
874 requestHeader := &ic.Header{
875 Id: uuid.New().String(),
876 Type: ic.MessageType_REQUEST,
877 FromTopic: replyTopic.Name,
878 ToTopic: toTopic.Name,
Matt Jeanneret384d8c92019-05-06 14:27:31 -0400879 KeyTopic: key,
Scott Bakered4a8e72020-04-17 11:10:20 -0700880 Timestamp: ptypes.TimestampNow(),
William Kurkianea869482019-04-09 15:16:11 -0400881 }
882 requestBody := &ic.InterContainerRequestBody{
883 Rpc: rpc,
884 ResponseRequired: true,
885 ReplyToTopic: replyTopic.Name,
886 }
887
888 for _, arg := range kvArgs {
889 if arg == nil {
890 // In case the caller sends an array with empty args
891 continue
892 }
893 var marshalledArg *any.Any
894 var err error
895 // ascertain the value interface type is a proto.Message
896 protoValue, ok := arg.Value.(proto.Message)
897 if !ok {
Esin Karamanccb714b2019-11-29 15:02:06 +0000898 logger.Warnw("argument-value-not-proto-message", log.Fields{"error": ok, "Value": arg.Value})
William Kurkianea869482019-04-09 15:16:11 -0400899 err := errors.New("argument-value-not-proto-message")
900 return nil, err
901 }
902 if marshalledArg, err = ptypes.MarshalAny(protoValue); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000903 logger.Warnw("cannot-marshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400904 return nil, err
905 }
906 protoArg := &ic.Argument{
907 Key: arg.Key,
908 Value: marshalledArg,
909 }
910 requestBody.Args = append(requestBody.Args, protoArg)
911 }
912
913 var marshalledData *any.Any
914 var err error
915 if marshalledData, err = ptypes.MarshalAny(requestBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000916 logger.Warnw("cannot-marshal-request", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400917 return nil, err
918 }
919 request := &ic.InterContainerMessage{
920 Header: requestHeader,
921 Body: marshalledData,
922 }
923 return request, nil
924}
925
926func decodeResponse(response *ic.InterContainerMessage) (*ic.InterContainerResponseBody, error) {
927 // Extract the message body
928 responseBody := ic.InterContainerResponseBody{}
929 if err := ptypes.UnmarshalAny(response.Body, &responseBody); err != nil {
Esin Karamanccb714b2019-11-29 15:02:06 +0000930 logger.Warnw("cannot-unmarshal-response", log.Fields{"error": err})
William Kurkianea869482019-04-09 15:16:11 -0400931 return nil, err
932 }
Esin Karamanccb714b2019-11-29 15:02:06 +0000933 //logger.Debugw("response-decoded-successfully", log.Fields{"response-status": &responseBody.Success})
William Kurkianea869482019-04-09 15:16:11 -0400934
935 return &responseBody, nil
936
937}