mirror of
https://github.com/go-micro/go-micro.git
synced 2025-06-12 22:07:47 +02:00
feat: add test framework & refactor RPC server (#2579)
Co-authored-by: Rene Jochum <rene@jochum.dev>
This commit is contained in:
@ -3,31 +3,42 @@ package client
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/pkg/errors"
|
||||
|
||||
"go-micro.dev/v4/broker"
|
||||
"go-micro.dev/v4/codec"
|
||||
raw "go-micro.dev/v4/codec/bytes"
|
||||
"go-micro.dev/v4/errors"
|
||||
merrors "go-micro.dev/v4/errors"
|
||||
log "go-micro.dev/v4/logger"
|
||||
"go-micro.dev/v4/metadata"
|
||||
"go-micro.dev/v4/registry"
|
||||
"go-micro.dev/v4/selector"
|
||||
"go-micro.dev/v4/transport"
|
||||
"go-micro.dev/v4/transport/headers"
|
||||
"go-micro.dev/v4/util/buf"
|
||||
"go-micro.dev/v4/util/net"
|
||||
"go-micro.dev/v4/util/pool"
|
||||
)
|
||||
|
||||
const (
|
||||
packageID = "go.micro.client"
|
||||
)
|
||||
|
||||
type rpcClient struct {
|
||||
seq uint64
|
||||
once atomic.Value
|
||||
opts Options
|
||||
pool pool.Pool
|
||||
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
func newRpcClient(opt ...Option) Client {
|
||||
func newRPCClient(opt ...Option) Client {
|
||||
opts := NewOptions(opt...)
|
||||
|
||||
p := pool.NewPool(
|
||||
@ -57,14 +68,17 @@ func (r *rpcClient) newCodec(contentType string) (codec.NewCodec, error) {
|
||||
if c, ok := r.opts.Codecs[contentType]; ok {
|
||||
return c, nil
|
||||
}
|
||||
|
||||
if cf, ok := DefaultCodecs[contentType]; ok {
|
||||
return cf, nil
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("unsupported Content-Type: %s", contentType)
|
||||
}
|
||||
|
||||
func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request, resp interface{}, opts CallOptions) error {
|
||||
address := node.Address
|
||||
logger := r.Options().Logger
|
||||
|
||||
msg := &transport.Message{
|
||||
Header: make(map[string]string),
|
||||
@ -73,31 +87,43 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
md, ok := metadata.FromContext(ctx)
|
||||
if ok {
|
||||
for k, v := range md {
|
||||
// don't copy Micro-Topic header, that used for pub/sub
|
||||
// this fix case then client uses the same context that received in subscriber
|
||||
if k == "Micro-Topic" {
|
||||
// Don't copy Micro-Topic header, that is used for pub/sub
|
||||
// this is fixes the case when the client uses the same context that
|
||||
// is received in the subscriber.
|
||||
if k == headers.Message {
|
||||
continue
|
||||
}
|
||||
|
||||
msg.Header[k] = v
|
||||
}
|
||||
}
|
||||
|
||||
// Set connection timeout for single requests to the server. Should be > 0
|
||||
// as otherwise requests can't be made.
|
||||
cTimeout := opts.ConnectionTimeout
|
||||
if cTimeout == 0 {
|
||||
logger.Log(log.DebugLevel, "connection timeout was set to 0, overridng to default connection timeout")
|
||||
|
||||
cTimeout = DefaultConnectionTimeout
|
||||
}
|
||||
|
||||
// set timeout in nanoseconds
|
||||
msg.Header["Timeout"] = fmt.Sprintf("%d", opts.RequestTimeout)
|
||||
msg.Header["Timeout"] = fmt.Sprintf("%d", cTimeout)
|
||||
// set the content type for the request
|
||||
msg.Header["Content-Type"] = req.ContentType()
|
||||
// set the accept header
|
||||
msg.Header["Accept"] = req.ContentType()
|
||||
|
||||
// setup old protocol
|
||||
cf := setupProtocol(msg, node)
|
||||
reqCodec := setupProtocol(msg, node)
|
||||
|
||||
// no codec specified
|
||||
if cf == nil {
|
||||
if reqCodec == nil {
|
||||
var err error
|
||||
cf, err = r.newCodec(req.ContentType())
|
||||
reqCodec, err = r.newCodec(req.ContentType())
|
||||
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.client", err.Error())
|
||||
return merrors.InternalServerError("go.micro.client", err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
@ -109,19 +135,29 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
dOpts = append(dOpts, transport.WithTimeout(opts.DialTimeout))
|
||||
}
|
||||
|
||||
if opts.ConnClose {
|
||||
dOpts = append(dOpts, transport.WithConnClose())
|
||||
}
|
||||
|
||||
c, err := r.pool.Get(address, dOpts...)
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.client", "connection error: %v", err)
|
||||
return merrors.InternalServerError("go.micro.client", "connection error: %v", err)
|
||||
}
|
||||
|
||||
seq := atomic.AddUint64(&r.seq, 1) - 1
|
||||
codec := newRpcCodec(msg, c, cf, "")
|
||||
codec := newRPCCodec(msg, c, reqCodec, "")
|
||||
|
||||
rsp := &rpcResponse{
|
||||
socket: c,
|
||||
codec: codec,
|
||||
}
|
||||
|
||||
releaseFunc := func(err error) {
|
||||
if err = r.pool.Release(c, err); err != nil {
|
||||
logger.Log(log.ErrorLevel, "failed to release pool", err)
|
||||
}
|
||||
}
|
||||
|
||||
stream := &rpcStream{
|
||||
id: fmt.Sprintf("%v", seq),
|
||||
context: ctx,
|
||||
@ -129,11 +165,17 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
response: rsp,
|
||||
codec: codec,
|
||||
closed: make(chan bool),
|
||||
release: func(err error) { r.pool.Release(c, err) },
|
||||
close: opts.ConnClose,
|
||||
release: releaseFunc,
|
||||
sendEOS: false,
|
||||
}
|
||||
|
||||
// close the stream on exiting this function
|
||||
defer stream.Close()
|
||||
defer func() {
|
||||
if err := stream.Close(); err != nil {
|
||||
logger.Log(log.ErrorLevel, "failed to close stream", err)
|
||||
}
|
||||
}()
|
||||
|
||||
// wait for error response
|
||||
ch := make(chan error, 1)
|
||||
@ -141,7 +183,7 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
go func() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
ch <- errors.InternalServerError("go.micro.client", "panic recovered: %v", r)
|
||||
ch <- merrors.InternalServerError("go.micro.client", "panic recovered: %v", r)
|
||||
}
|
||||
}()
|
||||
|
||||
@ -166,8 +208,8 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
select {
|
||||
case err := <-ch:
|
||||
return err
|
||||
case <-ctx.Done():
|
||||
grr = errors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
case <-time.After(cTimeout):
|
||||
grr = merrors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
}
|
||||
|
||||
// set the stream error
|
||||
@ -184,6 +226,7 @@ func (r *rpcClient) call(ctx context.Context, node *registry.Node, req Request,
|
||||
|
||||
func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request, opts CallOptions) (Stream, error) {
|
||||
address := node.Address
|
||||
logger := r.Options().Logger
|
||||
|
||||
msg := &transport.Message{
|
||||
Header: make(map[string]string),
|
||||
@ -206,14 +249,15 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
msg.Header["Accept"] = req.ContentType()
|
||||
|
||||
// set old codecs
|
||||
cf := setupProtocol(msg, node)
|
||||
nCodec := setupProtocol(msg, node)
|
||||
|
||||
// no codec specified
|
||||
if cf == nil {
|
||||
if nCodec == nil {
|
||||
var err error
|
||||
cf, err = r.newCodec(req.ContentType())
|
||||
|
||||
nCodec, err = r.newCodec(req.ContentType())
|
||||
if err != nil {
|
||||
return nil, errors.InternalServerError("go.micro.client", err.Error())
|
||||
return nil, merrors.InternalServerError("go.micro.client", err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
@ -227,7 +271,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
|
||||
c, err := r.opts.Transport.Dial(address, dOpts...)
|
||||
if err != nil {
|
||||
return nil, errors.InternalServerError("go.micro.client", "connection error: %v", err)
|
||||
return nil, merrors.InternalServerError("go.micro.client", "connection error: %v", err)
|
||||
}
|
||||
|
||||
// increment the sequence number
|
||||
@ -235,7 +279,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
id := fmt.Sprintf("%v", seq)
|
||||
|
||||
// create codec with stream id
|
||||
codec := newRpcCodec(msg, c, cf, id)
|
||||
codec := newRPCCodec(msg, c, nCodec, id)
|
||||
|
||||
rsp := &rpcResponse{
|
||||
socket: c,
|
||||
@ -247,6 +291,12 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
r.codec = codec
|
||||
}
|
||||
|
||||
releaseFunc := func(_ error) {
|
||||
if err = c.Close(); err != nil {
|
||||
logger.Log(log.ErrorLevel, err)
|
||||
}
|
||||
}
|
||||
|
||||
stream := &rpcStream{
|
||||
id: id,
|
||||
context: ctx,
|
||||
@ -257,8 +307,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
closed: make(chan bool),
|
||||
// signal the end of stream,
|
||||
sendEOS: true,
|
||||
// release func
|
||||
release: func(err error) { c.Close() },
|
||||
release: releaseFunc,
|
||||
}
|
||||
|
||||
// wait for error response
|
||||
@ -275,7 +324,7 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
case err := <-ch:
|
||||
grr = err
|
||||
case <-ctx.Done():
|
||||
grr = errors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
grr = merrors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
}
|
||||
|
||||
if grr != nil {
|
||||
@ -285,7 +334,10 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
stream.Unlock()
|
||||
|
||||
// close the stream
|
||||
stream.Close()
|
||||
if err := stream.Close(); err != nil {
|
||||
logger.Logf(log.ErrorLevel, "failed to close stream: %v", err)
|
||||
}
|
||||
|
||||
return nil, grr
|
||||
}
|
||||
|
||||
@ -293,6 +345,9 @@ func (r *rpcClient) stream(ctx context.Context, node *registry.Node, req Request
|
||||
}
|
||||
|
||||
func (r *rpcClient) Init(opts ...Option) error {
|
||||
r.mu.Lock()
|
||||
defer r.mu.Unlock()
|
||||
|
||||
size := r.opts.PoolSize
|
||||
ttl := r.opts.PoolTTL
|
||||
tr := r.opts.Transport
|
||||
@ -304,7 +359,10 @@ func (r *rpcClient) Init(opts ...Option) error {
|
||||
// update pool configuration if the options changed
|
||||
if size != r.opts.PoolSize || ttl != r.opts.PoolTTL || tr != r.opts.Transport {
|
||||
// close existing pool
|
||||
r.pool.Close()
|
||||
if err := r.pool.Close(); err != nil {
|
||||
return errors.Wrap(err, "failed to close pool")
|
||||
}
|
||||
|
||||
// create new pool
|
||||
r.pool = pool.NewPool(
|
||||
pool.Size(r.opts.PoolSize),
|
||||
@ -316,7 +374,11 @@ func (r *rpcClient) Init(opts ...Option) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Options retrives the options.
|
||||
func (r *rpcClient) Options() Options {
|
||||
r.mu.RLock()
|
||||
defer r.mu.RUnlock()
|
||||
|
||||
return r.opts
|
||||
}
|
||||
|
||||
@ -348,16 +410,22 @@ func (r *rpcClient) next(request Request, opts CallOptions) (selector.Next, erro
|
||||
// get next nodes from the selector
|
||||
next, err := r.opts.Selector.Select(service, opts.SelectOptions...)
|
||||
if err != nil {
|
||||
if err == selector.ErrNotFound {
|
||||
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
if errors.Is(err, selector.ErrNotFound) {
|
||||
return nil, merrors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
}
|
||||
return nil, errors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
|
||||
|
||||
return nil, merrors.InternalServerError("go.micro.client", "error selecting %s node: %s", service, err.Error())
|
||||
}
|
||||
|
||||
return next, nil
|
||||
}
|
||||
|
||||
func (r *rpcClient) Call(ctx context.Context, request Request, response interface{}, opts ...CallOption) error {
|
||||
// TODO: further validate these mutex locks. full lock would prevent
|
||||
// parallel calls. Maybe we can set individual locks for secctions.
|
||||
r.mu.RLock()
|
||||
defer r.mu.RUnlock()
|
||||
|
||||
// make a copy of call opts
|
||||
callOpts := r.opts.CallOptions
|
||||
for _, opt := range opts {
|
||||
@ -375,6 +443,7 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
// no deadline so we create a new one
|
||||
var cancel context.CancelFunc
|
||||
ctx, cancel = context.WithTimeout(ctx, callOpts.RequestTimeout)
|
||||
|
||||
defer cancel()
|
||||
} else {
|
||||
// got a deadline so no need to setup context
|
||||
@ -386,7 +455,7 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
// should we noop right here?
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return errors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
return merrors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
default:
|
||||
}
|
||||
|
||||
@ -403,7 +472,7 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
// call backoff first. Someone may want an initial start delay
|
||||
t, err := callOpts.Backoff(ctx, request, i)
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.client", "backoff error: %v", err.Error())
|
||||
return merrors.InternalServerError("go.micro.client", "backoff error: %v", err.Error())
|
||||
}
|
||||
|
||||
// only sleep if greater than 0
|
||||
@ -414,16 +483,19 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
// select next node
|
||||
node, err := next()
|
||||
service := request.Service()
|
||||
|
||||
if err != nil {
|
||||
if err == selector.ErrNotFound {
|
||||
return errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
if errors.Is(err, selector.ErrNotFound) {
|
||||
return merrors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
}
|
||||
return errors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
|
||||
|
||||
return merrors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
|
||||
}
|
||||
|
||||
// make the call
|
||||
err = rcall(ctx, node, request, response, callOpts)
|
||||
r.opts.Selector.Mark(service, node, err)
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
@ -431,11 +503,13 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
retries := callOpts.Retries
|
||||
|
||||
// disable retries when using a proxy
|
||||
if _, _, ok := net.Proxy(request.Service(), callOpts.Address); ok {
|
||||
retries = 0
|
||||
}
|
||||
// Note: I don't see why we should disable retries for proxies, so commenting out.
|
||||
// if _, _, ok := net.Proxy(request.Service(), callOpts.Address); ok {
|
||||
// retries = 0
|
||||
// }
|
||||
|
||||
ch := make(chan error, retries+1)
|
||||
|
||||
var gerr error
|
||||
|
||||
for i := 0; i <= retries; i++ {
|
||||
@ -445,7 +519,7 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return errors.Timeout("go.micro.client", fmt.Sprintf("call timeout: %v", ctx.Err()))
|
||||
return merrors.Timeout("go.micro.client", fmt.Sprintf("call timeout: %v", ctx.Err()))
|
||||
case err := <-ch:
|
||||
// if the call succeeded lets bail early
|
||||
if err == nil {
|
||||
@ -461,6 +535,8 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
return err
|
||||
}
|
||||
|
||||
r.opts.Logger.Logf(log.DebugLevel, "Retrying request. Previous attempt failed with: %v", err)
|
||||
|
||||
gerr = err
|
||||
}
|
||||
}
|
||||
@ -469,6 +545,9 @@ func (r *rpcClient) Call(ctx context.Context, request Request, response interfac
|
||||
}
|
||||
|
||||
func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOption) (Stream, error) {
|
||||
r.mu.RLock()
|
||||
defer r.mu.RUnlock()
|
||||
|
||||
// make a copy of call opts
|
||||
callOpts := r.opts.CallOptions
|
||||
for _, opt := range opts {
|
||||
@ -480,10 +559,9 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// should we noop right here?
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, errors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
return nil, merrors.Timeout("go.micro.client", fmt.Sprintf("%v", ctx.Err()))
|
||||
default:
|
||||
}
|
||||
|
||||
@ -491,7 +569,7 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
|
||||
// call backoff first. Someone may want an initial start delay
|
||||
t, err := callOpts.Backoff(ctx, request, i)
|
||||
if err != nil {
|
||||
return nil, errors.InternalServerError("go.micro.client", "backoff error: %v", err.Error())
|
||||
return nil, merrors.InternalServerError("go.micro.client", "backoff error: %v", err.Error())
|
||||
}
|
||||
|
||||
// only sleep if greater than 0
|
||||
@ -501,15 +579,18 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
|
||||
|
||||
node, err := next()
|
||||
service := request.Service()
|
||||
|
||||
if err != nil {
|
||||
if err == selector.ErrNotFound {
|
||||
return nil, errors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
if errors.Is(err, selector.ErrNotFound) {
|
||||
return nil, merrors.InternalServerError("go.micro.client", "service %s: %s", service, err.Error())
|
||||
}
|
||||
return nil, errors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
|
||||
|
||||
return nil, merrors.InternalServerError("go.micro.client", "error getting next %s node: %s", service, err.Error())
|
||||
}
|
||||
|
||||
stream, err := r.stream(ctx, node, request, callOpts)
|
||||
r.opts.Selector.Mark(service, node, err)
|
||||
|
||||
return stream, err
|
||||
}
|
||||
|
||||
@ -527,6 +608,7 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
|
||||
}
|
||||
|
||||
ch := make(chan response, retries+1)
|
||||
|
||||
var grr error
|
||||
|
||||
for i := 0; i <= retries; i++ {
|
||||
@ -537,7 +619,7 @@ func (r *rpcClient) Stream(ctx context.Context, request Request, opts ...CallOpt
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, errors.Timeout("go.micro.client", fmt.Sprintf("call timeout: %v", ctx.Err()))
|
||||
return nil, merrors.Timeout("go.micro.client", fmt.Sprintf("call timeout: %v", ctx.Err()))
|
||||
case rsp := <-ch:
|
||||
// if the call succeeded lets bail early
|
||||
if rsp.err == nil {
|
||||
@ -568,15 +650,15 @@ func (r *rpcClient) Publish(ctx context.Context, msg Message, opts ...PublishOpt
|
||||
o(&options)
|
||||
}
|
||||
|
||||
md, ok := metadata.FromContext(ctx)
|
||||
metadata, ok := metadata.FromContext(ctx)
|
||||
if !ok {
|
||||
md = make(map[string]string)
|
||||
metadata = make(map[string]string)
|
||||
}
|
||||
|
||||
id := uuid.New().String()
|
||||
md["Content-Type"] = msg.ContentType()
|
||||
md["Micro-Topic"] = msg.Topic()
|
||||
md["Micro-Id"] = id
|
||||
metadata["Content-Type"] = msg.ContentType()
|
||||
metadata[headers.Message] = msg.Topic()
|
||||
metadata[headers.ID] = id
|
||||
|
||||
// set the topic
|
||||
topic := msg.Topic()
|
||||
@ -589,7 +671,7 @@ func (r *rpcClient) Publish(ctx context.Context, msg Message, opts ...PublishOpt
|
||||
// encode message body
|
||||
cf, err := r.newCodec(msg.ContentType())
|
||||
if err != nil {
|
||||
return errors.InternalServerError("go.micro.client", err.Error())
|
||||
return merrors.InternalServerError(packageID, err.Error())
|
||||
}
|
||||
|
||||
var body []byte
|
||||
@ -598,33 +680,38 @@ func (r *rpcClient) Publish(ctx context.Context, msg Message, opts ...PublishOpt
|
||||
if d, ok := msg.Payload().(*raw.Frame); ok {
|
||||
body = d.Data
|
||||
} else {
|
||||
// new buffer
|
||||
b := buf.New(nil)
|
||||
|
||||
if err := cf(b).Write(&codec.Message{
|
||||
if err = cf(b).Write(&codec.Message{
|
||||
Target: topic,
|
||||
Type: codec.Event,
|
||||
Header: map[string]string{
|
||||
"Micro-Id": id,
|
||||
"Micro-Topic": msg.Topic(),
|
||||
headers.ID: id,
|
||||
headers.Message: msg.Topic(),
|
||||
},
|
||||
}, msg.Payload()); err != nil {
|
||||
return errors.InternalServerError("go.micro.client", err.Error())
|
||||
return merrors.InternalServerError(packageID, err.Error())
|
||||
}
|
||||
|
||||
// set the body
|
||||
body = b.Bytes()
|
||||
}
|
||||
|
||||
if !r.once.Load().(bool) {
|
||||
l, ok := r.once.Load().(bool)
|
||||
if !ok {
|
||||
return fmt.Errorf("failed to cast to bool")
|
||||
}
|
||||
|
||||
if !l {
|
||||
if err = r.opts.Broker.Connect(); err != nil {
|
||||
return errors.InternalServerError("go.micro.client", err.Error())
|
||||
return merrors.InternalServerError(packageID, err.Error())
|
||||
}
|
||||
|
||||
r.once.Store(true)
|
||||
}
|
||||
|
||||
return r.opts.Broker.Publish(topic, &broker.Message{
|
||||
Header: md,
|
||||
Header: metadata,
|
||||
Body: body,
|
||||
}, broker.PublishContext(options.Context))
|
||||
}
|
||||
|
Reference in New Issue
Block a user