Skip to content

Commit

Permalink
refactor registry & add direct service discovery through staticIP or …
Browse files Browse the repository at this point in the history
…DNS (#261)
  • Loading branch information
johnlanni authored Mar 29, 2023
1 parent 87366aa commit c96ede2
Show file tree
Hide file tree
Showing 8 changed files with 267 additions and 82 deletions.
25 changes: 20 additions & 5 deletions pkg/ingress/config/ingress_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -710,6 +710,7 @@ func (m *IngressConfig) convertIstioWasmPlugin(obj *higressext.WasmPlugin) (*ext
if !obj.DefaultConfigDisable {
result.PluginConfig = obj.DefaultConfig
}
hasValidRule := false
if len(obj.MatchRules) > 0 {
if result.PluginConfig == nil {
result.PluginConfig = &types.Struct{
Expand Down Expand Up @@ -769,14 +770,20 @@ func (m *IngressConfig) convertIstioWasmPlugin(obj *higressext.WasmPlugin) (*ext
Kind: v,
})
}
result.PluginConfig.Fields["_rules_"] = &types.Value{
Kind: &types.Value_ListValue{
ListValue: &types.ListValue{
Values: ruleValues,
if len(ruleValues) > 0 {
hasValidRule = true
result.PluginConfig.Fields["_rules_"] = &types.Value{
Kind: &types.Value_ListValue{
ListValue: &types.ListValue{
Values: ruleValues,
},
},
},
}
}
}
if !hasValidRule && obj.DefaultConfigDisable {
return nil, nil
}
return result, nil

}
Expand Down Expand Up @@ -807,6 +814,14 @@ func (m *IngressConfig) AddOrUpdateWasmPlugin(clusterNamespacedName util.Cluster
IngressLog.Errorf("invalid wasmPlugin:%s, err:%v", clusterNamespacedName.Name, err)
return
}
if istioWasmPlugin == nil {
IngressLog.Infof("wasmPlugin:%s will not be transferred to istio since config disabled",
clusterNamespacedName.Name)
m.mutex.Lock()
delete(m.wasmPlugins, clusterNamespacedName.Name)
m.mutex.Unlock()
return
}
IngressLog.Debugf("wasmPlugin:%s convert to istioWasmPlugin:%v", clusterNamespacedName.Name, istioWasmPlugin)
m.mutex.Lock()
m.wasmPlugins[clusterNamespacedName.Name] = istioWasmPlugin
Expand Down
164 changes: 164 additions & 0 deletions registry/direct/watcher.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,164 @@
// Copyright (c) 2022 Alibaba Group Holding Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package direct

import (
"net"
"regexp"
"strconv"
"strings"
"sync"

"istio.io/api/networking/v1alpha3"
"istio.io/istio/pkg/config/protocol"
"istio.io/pkg/log"

apiv1 "github.com/alibaba/higress/api/networking/v1"
"github.com/alibaba/higress/pkg/common"
"github.com/alibaba/higress/registry"
provider "github.com/alibaba/higress/registry"
"github.com/alibaba/higress/registry/memory"
)

type watcher struct {
provider.BaseWatcher
apiv1.RegistryConfig
cache memory.Cache
mutex sync.Mutex
}

type WatcherOption func(w *watcher)

func NewWatcher(cache memory.Cache, opts ...WatcherOption) (provider.Watcher, error) {
w := &watcher{
cache: cache,
}
for _, opt := range opts {
opt(w)
}
return w, nil
}

func WithType(t string) WatcherOption {
return func(w *watcher) {
w.Type = t
}
}

func WithName(name string) WatcherOption {
return func(w *watcher) {
w.Name = name
}
}

func WithDomain(domain string) WatcherOption {
return func(w *watcher) {
w.Domain = domain
}
}

func WithPort(port uint32) WatcherOption {
return func(w *watcher) {
w.Port = port
}
}

func (w *watcher) Run() {
w.mutex.Lock()
defer w.mutex.Unlock()
host := strings.Join([]string{w.Name, w.Type}, common.DotSeparator)
serviceEntry := w.generateServiceEntry(host)
if serviceEntry != nil {
w.cache.UpdateServiceEntryWrapper(host, &memory.ServiceEntryWrapper{
ServiceName: w.Name,
ServiceEntry: serviceEntry,
Suffix: w.Type,
RegistryType: w.Type,
})
w.UpdateService()
}
w.Ready(true)
}

func (w *watcher) Stop() {
w.mutex.Lock()
defer w.mutex.Unlock()
host := strings.Join([]string{w.Name, w.Type}, common.DotSeparator)
w.cache.DeleteServiceEntryWrapper(host)
w.Ready(false)
}

var domainRegex = regexp.MustCompile(`^(?:[a-zA-Z0-9](?:[a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?\.)+[a-zA-Z]{2,6}$`)

func (w *watcher) generateServiceEntry(host string) *v1alpha3.ServiceEntry {
endpoints := make([]*v1alpha3.WorkloadEntry, 0)
for _, ep := range strings.Split(w.Domain, common.CommaSeparator) {
var endpoint *v1alpha3.WorkloadEntry
if w.Type == string(registry.Static) {
pair := strings.Split(ep, common.ColonSeparator)
if len(pair) != 2 {
log.Errorf("invalid endpoint:%s with static type", ep)
return nil
}
port, err := strconv.ParseUint(pair[1], 10, 32)
if err != nil {
log.Errorf("invalid port:%s of endpoint:%s", pair[1], ep)
return nil
}
if net.ParseIP(pair[0]) == nil {
log.Errorf("invalid ip:%s of endpoint:%s", pair[0], ep)
return nil
}
endpoint = &v1alpha3.WorkloadEntry{
Address: pair[0],
Ports: map[string]uint32{"http": uint32(port)},
}
} else if w.Type == string(registry.DNS) {
if !domainRegex.MatchString(ep) {
log.Errorf("invalid domain format:%s", ep)
return nil
}
endpoint = &v1alpha3.WorkloadEntry{
Address: ep,
}
} else {
log.Errorf("unknown direct service type:%s", w.Type)
return nil
}
endpoints = append(endpoints, endpoint)
}
if len(endpoints) == 0 {
log.Errorf("empty endpoints will not be pushed, host:%s", host)
return nil
}
var ports []*v1alpha3.Port
ports = append(ports, &v1alpha3.Port{
Number: w.Port,
Name: "http",
Protocol: string(protocol.HTTP),
})
se := &v1alpha3.ServiceEntry{
Hosts: []string{host},
Ports: ports,
Location: v1alpha3.ServiceEntry_MESH_INTERNAL,
Endpoints: endpoints,
}
if w.Type == string(registry.Static) {
se.Resolution = v1alpha3.ServiceEntry_STATIC
} else if w.Type == string(registry.DNS) {
se.Resolution = v1alpha3.ServiceEntry_DNS
}
return se
}
42 changes: 31 additions & 11 deletions registry/memory/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
type Cache interface {
UpdateServiceEntryWrapper(service string, data *ServiceEntryWrapper)
DeleteServiceEntryWrapper(service string)
PurgeStaleService()
UpdateServiceEntryEnpointWrapper(service, ip, regionId, zoneId, protocol string, labels map[string]string)
GetServiceByEndpoints(requestVersions, endpoints map[string]bool, versionKey string, protocol common.Protocol) map[string][]string
GetAllServiceEntry() []*v1alpha3.ServiceEntry
Expand All @@ -39,20 +40,22 @@ type Cache interface {

func NewCache() Cache {
return &store{
mux: &sync.RWMutex{},
sew: make(map[string]*ServiceEntryWrapper),
toBeUpdated: make([]*ServiceEntryWrapper, 0),
toBeDeleted: make([]*ServiceEntryWrapper, 0),
ip2services: make(map[string]map[string]bool),
mux: &sync.RWMutex{},
sew: make(map[string]*ServiceEntryWrapper),
toBeUpdated: make([]*ServiceEntryWrapper, 0),
toBeDeleted: make([]*ServiceEntryWrapper, 0),
ip2services: make(map[string]map[string]bool),
deferedDelete: make(map[string]struct{}),
}
}

type store struct {
mux *sync.RWMutex
sew map[string]*ServiceEntryWrapper
toBeUpdated []*ServiceEntryWrapper
toBeDeleted []*ServiceEntryWrapper
ip2services map[string]map[string]bool
mux *sync.RWMutex
sew map[string]*ServiceEntryWrapper
toBeUpdated []*ServiceEntryWrapper
toBeDeleted []*ServiceEntryWrapper
ip2services map[string]map[string]bool
deferedDelete map[string]struct{}
}

func (s *store) UpdateServiceEntryEnpointWrapper(service, ip, regionId, zoneId, protocol string, labels map[string]string) {
Expand Down Expand Up @@ -105,6 +108,12 @@ func (s *store) UpdateServiceEntryWrapper(service string, data *ServiceEntryWrap

s.toBeUpdated = append(s.toBeUpdated, data)
s.sew[service] = data
// service is updated, should not be deleted
if _, ok := s.deferedDelete[service]; ok {
delete(s.deferedDelete, service)
log.Debugf("service in deferedDelete updated, host:%s", service)
}
log.Infof("ServiceEntry updated, host:%s", service)
}

func (s *store) DeleteServiceEntryWrapper(service string) {
Expand All @@ -114,7 +123,18 @@ func (s *store) DeleteServiceEntryWrapper(service string) {
if data, exist := s.sew[service]; exist {
s.toBeDeleted = append(s.toBeDeleted, data)
}
delete(s.sew, service)
s.deferedDelete[service] = struct{}{}
}

// should only be called when reconcile is done
func (s *store) PurgeStaleService() {
s.mux.Lock()
defer s.mux.Unlock()
for service := range s.deferedDelete {
delete(s.sew, service)
delete(s.deferedDelete, service)
log.Infof("ServiceEntry deleted, host:%s", service)
}
}

// GetServiceByEndpoints get the list of services of which "address:port" contained by the endpoints
Expand Down
20 changes: 5 additions & 15 deletions registry/nacos/v2/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,8 +60,6 @@ type watcher struct {
RegistryType provider.ServiceRegistryType `json:"registry_type"`
Status provider.WatcherStatus `json:"status"`
namingClient naming_client.INamingClient
updateHandler provider.ServiceUpdateHandler
readyHandler provider.ReadyHandler
cache memory.Cache
mutex *sync.Mutex
stop chan struct{}
Expand Down Expand Up @@ -234,7 +232,7 @@ func (w *watcher) Run() {
if err != nil {
log.Errorf("first fetch services failed, err:%v", err)
} else {
w.readyHandler(true)
w.Ready(true)
}
for {
select {
Expand All @@ -243,7 +241,7 @@ func (w *watcher) Run() {
if err != nil {
log.Errorf("fetch services failed, err:%v", err)
} else {
w.readyHandler(true)
w.Ready(true)
}
case <-w.stop:
return
Expand Down Expand Up @@ -399,7 +397,7 @@ func (w *watcher) getSubscribeCallback(groupName string, serviceName string) fun
host := strings.Join([]string{serviceName, suffix}, common.DotSeparator)

return func(services []model.Instance, err error) {
defer w.updateHandler()
defer w.UpdateService()

//log.Info("callback", "serviceName", serviceName, "suffix", suffix, "details", services)

Expand Down Expand Up @@ -484,8 +482,8 @@ func (w *watcher) Stop() {
}

w.isStop = true
w.stop <- struct{}{}
w.readyHandler(false)
close(w.stop)
w.Ready(false)
}

func (w *watcher) IsHealthy() bool {
Expand All @@ -496,14 +494,6 @@ func (w *watcher) GetRegistryType() string {
return w.RegistryType.String()
}

func (w *watcher) AppendServiceUpdateHandler(f func()) {
w.updateHandler = f
}

func (w *watcher) ReadyHandler(f func(bool)) {
w.readyHandler = f
}

func shouldSubscribe(serviceName string) bool {
prefixFilters := []string{"consumers:"}
fullFilters := []string{""}
Expand Down
Loading

0 comments on commit c96ede2

Please sign in to comment.