In the microservice architecture, there are many technical topics that cannot be avoided. Such as service discovery, load balancing, indicator monitoring, link tracking, and service governance related timeout control, circuit breaker, downgrade, current limit, etc., as well as RPC framework. These are the foundations of the microservice architecture. Only by laying down these foundations can you dare to say that you have a little understanding of microservices. You are embarrassed to say hello to others when you go out, and you can talk freely when you are asked questions. When problems arise, they can often trace the root cause and don’t panic. When the female colleague Xiaofang next to you looks at you, her eyes are full of cuteness.

In the official account of "Microservice Practice", I wrote a series of articles on "Go-zero Microservices Actual Combat Series". The articles in this series are more focused on business functions and service optimization under high concurrency. My level is limited, and it is inevitable that I have some shortcomings in writing, but I still have the support and encouragement from everyone. I feel very honored, so I decided to strike while the iron is hot, pursue the victory, and continue to output dry goods to everyone.

"Understanding Series" will learn all aspects of microservice architecture with you based on go-zero v1.3.5 and grpc-go v1.47.0 , the main form is theory + source code + case, if time permits, it may also add supporting videos .

As the first article in this series, this article will first introduce relatively simple service discovery related content.

Roll up your sleeves and start, Ollie! ! !

service discovery

Why do we need to introduce service discovery in a microservice architecture? In essence, the purpose of service discovery is to decouple the program's dependence on the specific location of the service. For microservice architecture, service discovery is not optional, but required. Because service providers in the production environment provide services in the form of clusters, the IP of services in the cluster may change at any time, such as service restart, release, expansion and contraction, etc., so we need to use a "address book" To obtain the corresponding service node in time, this process of obtaining is actually "service discovery".

To understand service discovery, you need to know that service discovery solves three problems:

  • Service Registration

    When the service starts, it should notify the service registration center of its own (service) information through some form (such as calling API, generating online event messages, recording in Etcd, storing database, etc.). The service framework is used to complete, and the business code is unaware.

  • Service Maintaining

    Although the offline mechanism is usually provided in the microservice framework, there is no way to ensure that each service can be gracefully offline (Graceful Shutdown), rather than suddenly disconnected due to downtime, network disconnection, etc. Therefore, in microservices The framework must ensure the correctness of the maintained service list as much as possible to avoid the embarrassment of accessing unavailable service nodes.

  • Service Discovery

    The discovery mentioned here is in a narrow sense. It specifically refers to the process that consumers convert a service identifier (usually a service name) into the actual location of the service (usually an ip address) from the microservice framework (service discovery module). This process (may be calling API, monitoring Etcd, querying the database, etc.) is unaware of the business code.

There are two modes of service discovery, namely server-side service discovery and client-side service discovery, which are described below.

Server-side service discovery

For server-side service discovery, the service caller does not need to pay attention to the specific details of service discovery, but only needs to know the DNS domain name of the service, and supports access in different languages. For infrastructure, it needs to specifically support load balancers. For the request link, there is one more network hop, which may result in performance loss. The nginx reverse proxy that we are familiar with can also be understood as server-side service discovery.

Client Service Discovery

For client-side service discovery, since the client and server use a direct connection, there is one less network jump than server-side service discovery. For the service caller, a built-in load balancer is required, and different languages require their own accomplish.

For the microservice architecture, we expect decentralized dependencies. Centralized dependencies will complicate the architecture, and when problems occur, the entire troubleshooting link will become cumbersome, so go-zero is adopted in go-zero is the mode of client service discovery.

Service discovery for gRPC

gRPC provides the ability to customize the Resolver to realize service discovery. Register the custom Resolver through the Register method. The custom Resolver needs to implement the Builder interface, which is defined as follows:

grpc-go/resolver/resolver.go:261

 type Builder interface {
    Build(target Target, cc ClientConn, opts BuildOptions) (Resolver, error)
    Scheme() string
}

Let's talk about the function of the Scheme() method, which returns a stirng. The registered Resolver will be stored in a global variable m, where m is a map, and the key of this map is the string returned by the Scheme() method. That is, multiple resolvers are distinguished by Scheme, so when we define Resolver Scheme do not repeat, otherwise Resolver will be overwritten.

grpc-go/resolver/resolver.go:49

 func Register(b Builder) {
    m[b.Scheme()] = b
}

Let's take a look at the Build method. The Build method has three parameters and the return value of the Resolver. At first glance, I don't know what these parameters are for. What should I do in this situation? In fact, it is very simple. Go to the source code to see where the Build method is called, and you will know where the incoming parameters come from and what they mean.

Before using gRPC to make a service call, you need to create a ClientConn object first. When the call is finally initiated, the Invoke method of ClientConn is actually called. You can see the following code, where ClientConn is passed in by calling NewGreeterClient , and NewGreeterClient is automatically generated for protoc code and assign it to the cc attribute. In the example code, the Dial method is called to create ClientConn , and the bottom layer will also call DialContext :

grpc-go/clientconn.go:104

 func Dial(target string, opts ...DialOption) (*ClientConn, error) {
    return DialContext(context.Background(), target, opts...)
}

Create a ClientConn object and pass it to the automatically generated greeterClient

grpc-go/examples/helloworld/greeter_client/main.go:42

 func main() {
    flag.Parse()

    // Set up a connection to the server.
    conn, err := grpc.Dial(*addr, grpc.WithTransportCredentials(insecure.NewCredentials()))
    if err != nil {
        log.Fatalf("did not connect: %v", err)
    }
    defer conn.Close()

    c := pb.NewGreeterClient(conn)
    // Contact the server and print out its response.
    ctx, cancel := context.WithTimeout(context.Background(), time.Second)
    defer cancel()
    r, err := c.SayHello(ctx, &pb.HelloRequest{Name: *name})
    if err != nil {
        log.Fatalf("could not greet: %v", err)
    }
    log.Printf("Greeting: %s", r.GetMessage())
}

Finally, the call request is actually initiated through the Invoke method.

grpc-go/examples/helloworld/helloworld/helloworld_grpc.pb.go:39

 func (c *greeterClient) SayHello(ctx context.Context, in *HelloRequest, opts ...grpc.CallOption) (*HelloReply, error) {
    out := new(HelloReply)
    err := c.cc.Invoke(ctx, "/helloworld.Greeter/SayHello", in, out, opts...)
    if err != nil {
        return nil, err
    }
    return out, nil
}

After understanding the process initiated by the client call, let's focus on the ClientConn method. This method is huge, and we only look at the Resolver part that we are concerned about. For the syntax of the second parameter Target of ClientConn , please refer to https://github.com/grpc/grpc/blob/master/doc/naming.md , which adopts the format of URI, where the first part represents the name of the Resolver, that is, the custom Builder The return value of the method Scheme. The format is as follows:

 dns:[//authority/]host[:port] -- DNS(默认)

Continue to look down, get the Resolver by calling the parseTargetAndFindResolver method

grpc-go/clientconn.go:251

 resolverBuilder, err := cc.parseTargetAndFindResolver()

In the parseTargetAndFindResolver method, the main thing is to parse the resolver name in the target , and then go to the global variable m that saves the Resolver mentioned above according to the resolver name to find the corresponding Resolver.

grpc-go/clientconn.go:1574

 func (cc *ClientConn) parseTargetAndFindResolver() (resolver.Builder, error) {
    // 非关键代码省略 ...
  
    var rb resolver.Builder
    parsedTarget, err := parseTarget(cc.target)
  
    // 非关键代码省略 ...
  
    rb = cc.getResolver(parsedTarget.Scheme)
    if rb == nil {
        return nil, fmt.Errorf("could not get resolver for default scheme: %q", parsedTarget.Scheme)
    }
    cc.parsedTarget = parsedTarget
    return rb, nil
}

Then look down, after finding our own registered Resolver, we called the newCCResolverWrapper method and passed our own Resolver in.

grpc-go/clientconn.go:292

 rWrapper, err := newCCResolverWrapper(cc, resolverBuilder)

Entering the newCCResolverWrapper method, in this method, we finally found where the Build method of our custom Builder was called. We called our custom Build method at grpc-go/resolver_conn_wrapper.go:72 , where the first parameter The target passed in is cc.parseTarget, and cc is the first parameter of newCCResolverWrapper , that is, the ClientConn object. cc.parseTarget is the last assignment in the above-mentioned custom Resolver method parseTargetAndFindResolver , in which Scheme, Authority, and Endpoint correspond to the three parts defined in the Target grammar. These attributes are about to be discarded, and only the URL attribute is retained. as follows:

grpc-go/resolver/resolver.go:245

 type Target struct {
    // Deprecated: use URL.Scheme instead.
    Scheme string
    // Deprecated: use URL.Host instead.
    Authority string
    // Deprecated: use URL.Path or URL.Opaque instead. The latter is set when
    // the former is empty.
    Endpoint string
    // URL contains the parsed dial target with an optional default scheme added
    // to it if the original dial target contained no scheme or contained an
    // unregistered scheme. Any query params specified in the original dial
    // target can be accessed from here.
    URL url.URL
}

The Scheme of the URL corresponds to the Scheme of the Target, the Host of the URL corresponds to the Authority of the Target, and the Path of the URL corresponds to the Endpoint of the Target

/usr/local/go/src/net/url/url.go:358

 type URL struct {
    Scheme      string
    Opaque      string    // encoded opaque data
    User        *Userinfo // username and password information
    Host        string    // host or host:port
    Path        string    // path (relative paths may omit leading slash)
    RawPath     string    // encoded path hint (see EscapedPath method)
    ForceQuery  bool      // append a query ('?') even if RawQuery is empty
    RawQuery    string    // encoded query values, without '?'
    Fragment    string    // fragment for references, without '#'
    RawFragment string    // encoded fragment hint (see EscapedFragment method)
}

Continue to look at the second parameter cc passed into the custom Build method. This cc parameter is an interface ClientConn , not to be confused with the ClientConn used to create the client call we talked about earlier. This ClientConn is defined as follows:

grpc-go/resolver/resolver.go:203

 type ClientConn interface {
    UpdateState(State) error
    ReportError(error)
    NewAddress(addresses []Address)
    NewServiceConfig(serviceConfig string)
    ParseServiceConfig(serviceConfigJSON string) *serviceconfig.ParseResult
}

ccResolverWrapper implements this interface and passes it in as the second parameter of the custom Build method

grpc-go/resolver_conn_wrapper.go:36

 type ccResolverWrapper struct {
    cc         *ClientConn
    resolverMu sync.Mutex
    resolver   resolver.Resolver
    done       *grpcsync.Event
    curState   resolver.State

    incomingMu sync.Mutex // Synchronizes all the incoming calls.
}

The third parameter of the custom Build method is some configuration items. The newCCResolverWrapper is implemented as follows:

grpc-go/resolver_conn_wrapper.go:48

 func newCCResolverWrapper(cc *ClientConn, rb resolver.Builder) (*ccResolverWrapper, error) {
    ccr := &ccResolverWrapper{
        cc:   cc,
        done: grpcsync.NewEvent(),
    }

    var credsClone credentials.TransportCredentials
    if creds := cc.dopts.copts.TransportCredentials; creds != nil {
        credsClone = creds.Clone()
    }
    rbo := resolver.BuildOptions{
        DisableServiceConfig: cc.dopts.disableServiceConfig,
        DialCreds:            credsClone,
        CredsBundle:          cc.dopts.copts.CredsBundle,
        Dialer:               cc.dopts.copts.Dialer,
    }

    var err error
    ccr.resolverMu.Lock()
    defer ccr.resolverMu.Unlock()
    ccr.resolver, err = rb.Build(cc.parsedTarget, ccr, rbo)
    if err != nil {
        return nil, err
    }
    return ccr, nil
}

Well, here we already know where the Build method of the custom Resolver is called, and the origin and meaning of the parameters passed in. If you are looking at the gRPC source code for the first time, you may be a little confused now, you can read more After several times, we provide you with a timing diagram for better reading with code:

How to implement service discovery in go-zero

Through the study of gRPC service discovery related content, we probably already know what is going on with service discovery. With the theory, let's take a look at how go-zero does service discovery based on gRPC.

From the above sequence diagram, you can see that the first step is to customize the Resolver, and the second step is to register the custom Resolver.

Go-zero's service discovery is implemented on the client side. When creating a zRPC client, a custom resolver is registered through the init method.

go-zero/zrpc/internal/client.go:23

 func init() {
    resolver.Register()
}

Four custom resolvers are registered by default in go-zero.

go-zero/zrpc/resolver/internal/resolver.go:35

 func RegisterResolver() {
    resolver.Register(&directResolverBuilder)
    resolver.Register(&discovResolverBuilder)
    resolver.Register(&etcdResolverBuilder)
    resolver.Register(&k8sResolverBuilder)
}

The rpc code automatically generated by goctl uses etcd as the service registration and discovery component by default, so let's focus on how go-zero implements service registration and discovery based on etcd.

The Scheme value returned by etcdBuilder is etcd

go-zero/zrpc/resolver/internal/etcdbuilder.go:7

 func (b *etcdBuilder) Scheme() string {
    return EtcdScheme
}

go-zero/zrpc/resolver/internal/resolver.go:15

 EtcdScheme = "etcd"

Remember what we said above? In the fifth and sixth steps of the sequence diagram, the custom Resolver will be found in the global m through the scheme, and the scheme is parsed from the second parameter target of DialContext, then let's see that go-zero calls DialContext When , what is the incoming target value. The target is obtained through the BuildTarget method and is defined as follows:

go-zero/zrpc/config.go:72

 func (cc RpcClientConf) BuildTarget() (string, error) {
    if len(cc.Endpoints) > 0 {
        return resolver.BuildDirectTarget(cc.Endpoints), nil
    } else if len(cc.Target) > 0 {
        return cc.Target, nil
    }

    if err := cc.Etcd.Validate(); err != nil {
        return "", err
    }

    if cc.Etcd.HasAccount() {
        discov.RegisterAccount(cc.Etcd.Hosts, cc.Etcd.User, cc.Etcd.Pass)
    }
    if cc.Etcd.HasTLS() {
        if err := discov.RegisterTLS(cc.Etcd.Hosts, cc.Etcd.CertFile, cc.Etcd.CertKeyFile,
            cc.Etcd.CACertFile, cc.Etcd.InsecureSkipVerify); err != nil {
            return "", err
        }
    }

    return resolver.BuildDiscovTarget(cc.Etcd.Hosts, cc.Etcd.Key), nil
}

The final method of generating the target result is as follows, that is, for etcd, the final generated target format is:

 etcd://127.0.0.1:2379/product.rpc

go-zero/zrpc/resolver/target.go:17

 func BuildDiscovTarget(endpoints []string, key string) string {
    return fmt.Sprintf("%s://%s/%s", internal.DiscovScheme,
        strings.Join(endpoints, internal.EndpointSep), key)
}

Something seems wrong, shouldn't the scheme be etcd? Why discov? In fact, it is because etcd and discov share a set of Resolver logic, that is, gRPC finds the registered discov Resolver through the scheme. The Build method corresponding to the Resolver is also applicable to etcd. Discov can be considered as an abstraction of service discovery, the definition of etcdResolver as follows:

go-zero/zrpc/resolver/internal/etcdbuilder.go:3

 type etcdBuilder struct {
    discovBuilder
}

service registration

Before looking at etcd-based custom Resolver logic in detail, let's take a look at go-zero's service registration, that is, how to register service information into etcd. Let's take the service lebron /apps/product/rpc as an example to illustrate .

Etcd is configured in the configuration file of product-rpc, including the address of etcd and the key corresponding to the service, as follows:

lebron/apps/product/rpc/etc/product.yaml:4

 ListenOn: 127.0.0.1:9002

Etcd:
  Hosts:
  - 127.0.0.1:2379
  Key: product.rpc

Call zrpc.MustNewServer to create a gRPC server, and then call the NewRpcPubServer method, which is defined as follows:

go-zero/zrpc/internal/rpcpubserver.go:17

 func NewRpcPubServer(etcd discov.EtcdConf, listenOn string, opts ...ServerOption) (Server, error) {
    registerEtcd := func() error {
        pubListenOn := figureOutListenOn(listenOn)
        var pubOpts []discov.PubOption
        if etcd.HasAccount() {
            pubOpts = append(pubOpts, discov.WithPubEtcdAccount(etcd.User, etcd.Pass))
        }
        if etcd.HasTLS() {
            pubOpts = append(pubOpts, discov.WithPubEtcdTLS(etcd.CertFile, etcd.CertKeyFile,
                etcd.CACertFile, etcd.InsecureSkipVerify))
        }
        pubClient := discov.NewPublisher(etcd.Hosts, etcd.Key, pubListenOn, pubOpts...)
        return pubClient.KeepAlive()
    }
    server := keepAliveServer{
        registerEtcd: registerEtcd,
        Server:       NewRpcServer(listenOn, opts...),
    }

    return server, nil
}

When starting the Server, the Start method is called, and registerEtcd is called in the Start method for real service registration

go-zero/zrpc/internal/rpcpubserver.go:44

 func (s keepAliveServer) Start(fn RegisterFn) error {
    if err := s.registerEtcd(); err != nil {
        return err
    }

    return s.Server.Start(fn)
}

In the KeepAlive method, first create the etcd connection, then call the register method to register the service, first create a lease in the register, the lease default time is 10 seconds, and finally register through the Put method.

go-zero/core/discov/publisher.go:125

 func (p *Publisher) register(client internal.EtcdClient) (clientv3.LeaseID, error) {
    resp, err := client.Grant(client.Ctx(), TimeToLive)
    if err != nil {
        return clientv3.NoLease, err
    }

    lease := resp.ID
    if p.id > 0 {
        p.fullKey = makeEtcdKey(p.key, p.id)
    } else {
        p.fullKey = makeEtcdKey(p.key, int64(lease))
    }
    _, err = client.Put(client.Ctx(), p.fullKey, p.value, clientv3.WithLease(lease))

    return lease, err
}

The key rules are defined as follows, where key is the key configured in the configuration file, here is product.rpc, and id is the lease id. value is the address of the service.

go-zero/core/discov/clients.go:39

 func makeEtcdKey(key string, id int64) string {
    return fmt.Sprintf("%s%c%d", key, internal.Delimiter, id)
}

After understanding the process of service registration, we start the product-rpc service, and then use the following command to view the service registration address:

 $ etcdctl get product.rpc --prefix
product.rpc/7587864068988009477
127.0.0.1:9002

In the KeepAlive method, after the service is registered, keepAliveAsync is called to renew the lease to ensure that the service is always alive. If the service exits abnormally, the renewal cannot be performed, and service discovery can also be automatically identified. The service went offline abnormally.

service discovery

Now that the service has been registered with etcd, let's continue to see how to discover these service addresses. We return to the implementation of etcdBuilder 's Build method.

Remember what the first parameter target is? If you don't remember, you can go up and review it again. First, parse the etcd address from the target and the key corresponding to the service. Then create the etcd connection, and then execute the update method. In the update method, the service state is updated by calling the cc.UpdateState method.

go-zero/zrpc/resolver/internal/discovbuilder.go:14

 func (b *discovBuilder) Build(target resolver.Target, cc resolver.ClientConn, _ resolver.BuildOptions) (
    resolver.Resolver, error) {
    hosts := strings.FieldsFunc(targets.GetAuthority(target), func(r rune) bool {
        return r == EndpointSepChar
    })
    sub, err := discov.NewSubscriber(hosts, targets.GetEndpoints(target))
    if err != nil {
        return nil, err
    }

    update := func() {
        var addrs []resolver.Address
        for _, val := range subset(sub.Values(), subsetSize) {
            addrs = append(addrs, resolver.Address{
                Addr: val,
            })
        }
        if err := cc.UpdateState(resolver.State{
            Addresses: addrs,
        }); err != nil {
            logx.Error(err)
        }
    }
    sub.AddListener(update)
    update()

    return &nopResolver{cc: cc}, nil
}

If you forget the second parameter cc of the Build method, you can look up and review it again. The cc.UpdateState method is defined as follows, and will eventually call the updateResolverState method of ClientConn :

grpc-go/resolver_conn_wrapper.go:94

 func (ccr *ccResolverWrapper) UpdateState(s resolver.State) error {
    ccr.incomingMu.Lock()
    defer ccr.incomingMu.Unlock()
    if ccr.done.HasFired() {
        return nil
    }
    ccr.addChannelzTraceEvent(s)
    ccr.curState = s
    if err := ccr.cc.updateResolverState(ccr.curState, nil); err == balancer.ErrBadResolverState {
        return balancer.ErrBadResolverState
    }
    return nil
}

Continue to look at the Build method. The update method will be added to the event monitoring. When the PUT and DELETE events are triggered, the update method will be called to update the service status. The event monitoring is implemented through the Watch mechanism of etcd. The code is as follows:

go-zero/core/discov/internal/registry.go:295

 func (c *cluster) watchStream(cli EtcdClient, key string) bool {
    rch := cli.Watch(clientv3.WithRequireLeader(c.context(cli)), makeKeyPrefix(key), clientv3.WithPrefix())
    for {
        select {
        case wresp, ok := <-rch:
            if !ok {
                logx.Error("etcd monitor chan has been closed")
                return false
            }
            if wresp.Canceled {
                logx.Errorf("etcd monitor chan has been canceled, error: %v", wresp.Err())
                return false
            }
            if wresp.Err() != nil {
                logx.Error(fmt.Sprintf("etcd monitor chan error: %v", wresp.Err()))
                return false
            }

            c.handleWatchEvents(key, wresp.Events)
        case <-c.done:
            return true
        }
    }
}

When an event is triggered, the event handler handleWatchEvents will be called, and finally the update defined in the Build method will be called to update the service status:

go-zero/core/discov/internal/registry.go:172

 func (c *cluster) handleWhandleWatchEventsatchEvents(key string, events []*clientv3.Event) {
    c.lock.Lock()
    listeners := append([]UpdateListener(nil), c.listeners[key]...)
    c.lock.Unlock()

    for _, ev := range events {
        switch ev.Type {
        case clientv3.EventTypePut:
            c.lock.Lock()
            if vals, ok := c.values[key]; ok {
                vals[string(ev.Kv.Key)] = string(ev.Kv.Value)
            } else {
                c.values[key] = map[string]string{string(ev.Kv.Key): string(ev.Kv.Value)}
            }
            c.lock.Unlock()
            for _, l := range listeners {
                l.OnAdd(KV{
                    Key: string(ev.Kv.Key),
                    Val: string(ev.Kv.Value),
                })
            }
        case clientv3.EventTypeDelete:
            c.lock.Lock()
            if vals, ok := c.values[key]; ok {
                delete(vals, string(ev.Kv.Key))
            }
            c.lock.Unlock()
            for _, l := range listeners {
                l.OnDelete(KV{
                    Key: string(ev.Kv.Key),
                    Val: string(ev.Kv.Value),
                })
            }
        default:
            logx.Errorf("Unknown event type: %v", ev.Type)
        }
    }
}

For the first time, the load method will be called to obtain the service list corresponding to the key, which is obtained by matching the etcd prefix. The obtaining method is as follows:

 func (c *cluster) load(cli EtcdClient, key string) {
    var resp *clientv3.GetResponse
    for {
        var err error
        ctx, cancel := context.WithTimeout(c.context(cli), RequestTimeout)
        resp, err = cli.Get(ctx, makeKeyPrefix(key), clientv3.WithPrefix())
        cancel()
        if err == nil {
            break
        }

        logx.Error(err)
        time.Sleep(coolDownInterval)
    }

    var kvs []KV
    for _, ev := range resp.Kvs {
        kvs = append(kvs, KV{
            Key: string(ev.Key),
            Val: string(ev.Value),
        })
    }

    c.handleChanges(key, kvs)
}

The obtained service address list is stored locally through the map. When an event is triggered, the service list is updated by operating the map. There is a hidden design consideration here. When etcd cannot be connected or fails, the service address in memory The list will not be updated, which ensures that when etcd has problems, service discovery can still work, and the service continues to run normally. The logic is relatively intuitive, so I won’t repeat it here. The code logic is in go-zero/core/discov/subscriber.go:76 , and the following is the sequence diagram of go-zero service discovery

concluding remarks

At this point, the service found that the relevant content has been finished, and the content is still a bit too much, especially the code part needs to be read carefully to deepen the understanding.

Let's briefly review the content of this article:

  • First, the concept of service discovery and what problems need to be solved by service discovery are introduced
  • There are two modes of service discovery: server-side discovery mode and client-side discovery mode.
  • Then we learned the ability to register Resolver provided by gRPC, realize custom service discovery function by registering Resolver, and how to find custom Resolver inside gRPC and trigger the logic of calling custom Resolver
  • Finally, I learned the implementation principle of service discovery in go-zero,

    • First, the service registration process of go-zero is introduced, and the effect of the final registration is demonstrated.
    • Then, starting from the Build method of the custom Resolver, I learned that the corresponding service list is obtained by prefix matching and exists locally, and then the UpdateState method is called to update the service state.
    • The changes in the service status are monitored by means of Watch. After monitoring the changes, the update method will be called to update the local service list and UpdateState will be called to update the service status.

Service discovery is the basis for understanding the microservice architecture. I hope you can read this article carefully. If you have any questions, you can feel free to discuss with me. You can search for dawn_zhou in the community group to find me.

After the service list is obtained through service discovery, the service will be invoked through the Invoke method. Load balancing is involved in the service invocation, and a suitable node is selected through load balancing to initiate the request. Load balancing is the content of the next article, so stay tuned.

I hope this article is helpful to you. Your likes are the biggest motivation for the author to continue to output.

project address

https://github.com/zeromicro/go-zero

Welcome go-zero and star support us!

WeChat exchange group

Follow the official account of " Microservice Practice " and click on the exchange group to get the QR code of the community group.


kevinwan
931 声望3.5k 粉丝

go-zero作者