diff --git a/Dockerfile b/Dockerfile index 526fa2f8..26c76087 100644 --- a/Dockerfile +++ b/Dockerfile @@ -30,3 +30,4 @@ ENTRYPOINT ["/gubernator"] EXPOSE 80 EXPOSE 81 +EXPOSE 7946 diff --git a/benchmark_test.go b/benchmark_test.go index 70e2318d..ac45c96f 100644 --- a/benchmark_test.go +++ b/benchmark_test.go @@ -31,10 +31,7 @@ func BenchmarkServer_GetPeerRateLimitNoBatching(b *testing.B) { b.Errorf("SetDefaults err: %s", err) } - client, err := guber.NewPeerClient(conf.Behaviors, cluster.GetPeer()) - if err != nil { - b.Errorf("NewPeerClient err: %s", err) - } + client := guber.NewPeerClient(conf.Behaviors, cluster.GetRandomPeer()) b.Run("GetPeerRateLimitNoBatching", func(b *testing.B) { for n := 0; n < b.N; n++ { @@ -54,7 +51,7 @@ func BenchmarkServer_GetPeerRateLimitNoBatching(b *testing.B) { } func BenchmarkServer_GetRateLimit(b *testing.B) { - client, err := guber.DialV1Server(cluster.GetPeer()) + client, err := guber.DialV1Server(cluster.GetRandomPeer().Address) if err != nil { b.Errorf("NewV1Client err: %s", err) } @@ -80,7 +77,7 @@ func BenchmarkServer_GetRateLimit(b *testing.B) { } func BenchmarkServer_Ping(b *testing.B) { - client, err := guber.DialV1Server(cluster.GetPeer()) + client, err := guber.DialV1Server(cluster.GetRandomPeer().Address) if err != nil { b.Errorf("NewV1Client err: %s", err) } @@ -108,7 +105,7 @@ func BenchmarkServer_Ping(b *testing.B) { }*/ func BenchmarkServer_ThunderingHeard(b *testing.B) { - client, err := guber.DialV1Server(cluster.GetPeer()) + client, err := guber.DialV1Server(cluster.GetRandomPeer().Address) if err != nil { b.Errorf("NewV1Client err: %s", err) } diff --git a/client.go b/client.go index d859a38d..fbe6aac7 100644 --- a/client.go +++ b/client.go @@ -63,7 +63,7 @@ func FromUnixMilliseconds(ts int64) time.Time { } // Given a list of peers, return a random peer -func RandomPeer(peers []string) string { +func RandomPeer(peers []PeerInfo) PeerInfo { return peers[rand.Intn(len(peers))] } diff --git a/cluster/cluster.go b/cluster/cluster.go index 8a889341..af9b4bae 100644 --- a/cluster/cluster.go +++ b/cluster/cluster.go @@ -36,11 +36,10 @@ type instance struct { func (i *instance) Peers() []gubernator.PeerInfo { var result []gubernator.PeerInfo for _, peer := range peers { - info := gubernator.PeerInfo{Address: peer} - if peer == i.Address { - info.IsOwner = true + if peer.Address == i.Address { + peer.IsOwner = true } - result = append(result, info) + result = append(result, peer) } return result } @@ -52,15 +51,27 @@ func (i *instance) Stop() error { } var instances []*instance -var peers []string +var peers []gubernator.PeerInfo + +// Returns default testing configuration +func GetDefaultConfig() gubernator.Config { + return gubernator.Config{ + Behaviors: gubernator.BehaviorConfig{ + GlobalSyncWait: time.Millisecond * 50, // Suitable for testing but not production + GlobalTimeout: time.Second, + MultiRegionSyncWait: time.Millisecond * 50, // Suitable for testing but not production + MultiRegionTimeout: time.Second, + }, + } +} // Returns a random peer from the cluster -func GetPeer() string { +func GetRandomPeer() gubernator.PeerInfo { return gubernator.RandomPeer(peers) } // Returns a specific peer -func PeerAt(idx int) string { +func PeerAt(idx int) gubernator.PeerInfo { return peers[idx] } @@ -79,6 +90,16 @@ func InstanceForHost(host string) *instance { return nil } +// Stop an instance without updating peers, used to cause connection errors +func StopInstanceAt(idx int) { + instances[idx].Stop() +} + +// Returns the number of instances +func NumOfInstances() int { + return len(instances) +} + // Start a local cluster of gubernator servers func Start(numInstances int) error { addresses := make([]string, numInstances, numInstances) @@ -87,19 +108,15 @@ func Start(numInstances int) error { // Start a local cluster with specific addresses func StartWith(addresses []string) error { + config := GetDefaultConfig() for _, address := range addresses { - ins, err := StartInstance(address, gubernator.Config{ - Behaviors: gubernator.BehaviorConfig{ - GlobalSyncWait: time.Millisecond * 50, // Suitable for testing but not production - GlobalTimeout: time.Second, - }, - }) + ins, err := StartInstance(address, config) if err != nil { return errors.Wrapf(err, "while starting instance for addr '%s'", address) } // Add the peers and instances to the package level variables - peers = append(peers, ins.Address) + peers = append(peers, gubernator.PeerInfo{Address: ins.Address}) instances = append(instances, ins) } diff --git a/cluster/cluster_test.go b/cluster/cluster_test.go index e7be6327..9658cfef 100644 --- a/cluster/cluster_test.go +++ b/cluster/cluster_test.go @@ -27,13 +27,13 @@ func Test_instance_Peers(t *testing.T) { tests := []struct { name string instance *instance - peers []string + peers []gubernator.PeerInfo want []gubernator.PeerInfo }{ { name: "Happy path", instance: &instance{Address: "mailgun.com"}, - peers: []string{"mailgun.com"}, + peers: []gubernator.PeerInfo{{Address: "mailgun.com"}}, want: []gubernator.PeerInfo{ {Address: "mailgun.com", IsOwner: true}, }, @@ -41,7 +41,7 @@ func Test_instance_Peers(t *testing.T) { { name: "Get multy peers", instance: &instance{Address: "mailgun.com"}, - peers: []string{"localhost:11111", "mailgun.com"}, + peers: []gubernator.PeerInfo{{Address: "localhost:11111"}, {Address: "mailgun.com"}}, want: []gubernator.PeerInfo{ {Address: "localhost:11111"}, {Address: "mailgun.com", IsOwner: true}, @@ -50,7 +50,7 @@ func Test_instance_Peers(t *testing.T) { { name: "No Peers", instance: &instance{Address: "www.mailgun.com:11111"}, - peers: []string{}, + peers: []gubernator.PeerInfo{}, want: []gubernator.PeerInfo(nil), }, { @@ -62,7 +62,7 @@ func Test_instance_Peers(t *testing.T) { { name: "Owner does not exist", instance: &instance{Address: "mailgun.com"}, - peers: []string{"localhost:11111"}, + peers: []gubernator.PeerInfo{{Address: "localhost:11111"}}, want: []gubernator.PeerInfo{ {Address: "localhost:11111"}, }, @@ -82,36 +82,33 @@ func Test_instance_Peers(t *testing.T) { func TestGetPeer(t *testing.T) { tests := []struct { name string - peers []string + peers []gubernator.PeerInfo oneOf map[string]bool }{ { name: "Happy path", - peers: []string{"mailgun.com"}, + peers: []gubernator.PeerInfo{{Address: "mailgun.com"}}, }, { name: "Get one peer from multiple peers", - peers: []string{"mailgun.com", "localhost", "test.com"}, + peers: []gubernator.PeerInfo{{Address: "mailgun.com"}, {Address: "localhost"}, {Address: "test.com"}}, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { peers = tt.peers - got := GetPeer() + got := GetRandomPeer() - if !stringInSlice(got, peers) { - t.Errorf("expected one of: %v", tt.peers) - t.Errorf("got: %s", got) - } + assert.Contains(t, peers, got) }) } } func TestPeerAt(t *testing.T) { - peers = []string{"mailgun.com"} + peers = []gubernator.PeerInfo{{Address: "mailgun.com"}} got := PeerAt(0) - want := "mailgun.com" + want := gubernator.PeerInfo{Address: "mailgun.com"} assert.Equal(t, want, got) } @@ -186,7 +183,7 @@ func TestStartMultipleInstancesWithAddresses(t *testing.T) { err := StartWith(addresses) assert.Nil(t, err) - wantPeers := []string{"127.0.0.1:11111", "127.0.0.1:22222"} + wantPeers := []gubernator.PeerInfo{{Address: "127.0.0.1:11111"}, {Address: "127.0.0.1:22222"}} wantInstances := []*instance{ {Address: "127.0.0.1:11111"}, {Address: "127.0.0.1:22222"}, diff --git a/cmd/gubernator/config.go b/cmd/gubernator/config.go index a538d073..36987edf 100644 --- a/cmd/gubernator/config.go +++ b/cmd/gubernator/config.go @@ -46,6 +46,7 @@ type ServerConfig struct { HTTPListenAddress string EtcdKeyPrefix string CacheSize int + DataCenter string // Etcd configuration used to find peers EtcdConf etcd.Config @@ -56,6 +57,9 @@ type ServerConfig struct { // K8s configuration used to find peers inside a K8s cluster K8PoolConf gubernator.K8sPoolConfig + // Memberlist configuration used to find peers + MemberlistPoolConf gubernator.MemberlistPoolConfig + // The PeerPicker as selected by `GUBER_PEER_PICKER` Picker gubernator.PeerPicker } @@ -95,6 +99,7 @@ func confFromEnv() (ServerConfig, error) { setter.SetDefault(&conf.GRPCListenAddress, os.Getenv("GUBER_GRPC_ADDRESS"), "0.0.0.0:81") setter.SetDefault(&conf.HTTPListenAddress, os.Getenv("GUBER_HTTP_ADDRESS"), "0.0.0.0:80") setter.SetDefault(&conf.CacheSize, getEnvInteger("GUBER_CACHE_SIZE"), 50000) + setter.SetDefault(&conf.DataCenter, os.Getenv("GUBER_DATA_CENTER"), "") // Behaviors setter.SetDefault(&conf.Behaviors.BatchTimeout, getEnvDuration("GUBER_BATCH_TIMEOUT")) @@ -105,6 +110,10 @@ func confFromEnv() (ServerConfig, error) { setter.SetDefault(&conf.Behaviors.GlobalBatchLimit, getEnvInteger("GUBER_GLOBAL_BATCH_LIMIT")) setter.SetDefault(&conf.Behaviors.GlobalSyncWait, getEnvDuration("GUBER_GLOBAL_SYNC_WAIT")) + setter.SetDefault(&conf.Behaviors.MultiRegionTimeout, getEnvDuration("GUBER_MULTI_REGION_TIMEOUT")) + setter.SetDefault(&conf.Behaviors.MultiRegionBatchLimit, getEnvInteger("GUBER_MULTI_REGION_BATCH_LIMIT")) + setter.SetDefault(&conf.Behaviors.MultiRegionSyncWait, getEnvDuration("GUBER_MULTI_REGION_SYNC_WAIT")) + // ETCD Config setter.SetDefault(&conf.EtcdAdvertiseAddress, os.Getenv("GUBER_ETCD_ADVERTISE_ADDRESS"), "127.0.0.1:81") setter.SetDefault(&conf.EtcdKeyPrefix, os.Getenv("GUBER_ETCD_KEY_PREFIX"), "/gubernator-peers") @@ -113,6 +122,11 @@ func confFromEnv() (ServerConfig, error) { setter.SetDefault(&conf.EtcdConf.Username, os.Getenv("GUBER_ETCD_USER")) setter.SetDefault(&conf.EtcdConf.Password, os.Getenv("GUBER_ETCD_PASSWORD")) + // Memberlist Config + setter.SetDefault(&conf.MemberlistPoolConf.AdvertiseAddress, os.Getenv("GUBER_MEMBERLIST_ADVERTISE_ADDRESS"), "") + setter.SetDefault(&conf.MemberlistPoolConf.AdvertisePort, os.Getenv("GUBER_MEMBERLIST_ADVERTISE_PORT"), 7946) + setter.SetDefault(&conf.MemberlistPoolConf.KnownNodes, getEnvSlice("GUBER_MEMBERLIST_KNOWN_NODES"), []string{}) + // Kubernetes Config setter.SetDefault(&conf.K8PoolConf.Namespace, os.Getenv("GUBER_K8S_NAMESPACE"), "default") conf.K8PoolConf.PodIP = os.Getenv("GUBER_K8S_POD_IP") @@ -163,11 +177,25 @@ func confFromEnv() (ServerConfig, error) { } } + if anyHasPrefix("GUBER_MEMBERLIST_", os.Environ()) { + logrus.Debug("Memberlist pool config found") + conf.MemberlistPoolConf.Enabled = true + if conf.K8PoolConf.Enabled { + return conf, errors.New("refusing to register gubernator peers with both memberlist and k8s;" + + " remove either `GUBER_MEMBERLIST_*` or `GUBER_K8S_*` variables from the environment") + } + + if len(conf.MemberlistPoolConf.KnownNodes) == 0 { + return conf, errors.New("when using memberlist for peer discovery, you MUST provide a " + + "hostname of a known host in the cluster via `GUBER_MEMBERLIST_KNOWN_NODES`") + } + } + if anyHasPrefix("GUBER_ETCD_", os.Environ()) { logrus.Debug("ETCD peer pool config found") - if conf.K8PoolConf.Enabled { - return conf, errors.New("refusing to register gubernator peers with both etcd and k8s;" + - " remove either `GUBER_ETCD_*` or `GUBER_K8S_*` variables from the environment") + if conf.K8PoolConf.Enabled || conf.MemberlistPoolConf.Enabled { + return conf, errors.New("refusing to register gubernator peers with both etcd, memberlist and k8s;" + + " remove all but one of `GUBER_MEMBERLIST_*`, `GUBER_ETCD_*` or `GUBER_K8S_*` variables from the environment") } } diff --git a/cmd/gubernator/main.go b/cmd/gubernator/main.go index 5746f8b4..afb06cb7 100644 --- a/cmd/gubernator/main.go +++ b/cmd/gubernator/main.go @@ -22,10 +22,12 @@ import ( "net/http" "os" "os/signal" + "strconv" + "strings" "github.com/grpc-ecosystem/grpc-gateway/runtime" "github.com/mailgun/gubernator" - "github.com/mailgun/holster/v3/etcdutil" + "github.com/mailgun/holster/etcdutil" "github.com/mailgun/holster/v3/syncutil" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" @@ -61,9 +63,10 @@ func main() { // Registers a new gubernator instance with the GRPC server guber, err := gubernator.New(gubernator.Config{ - Picker: conf.Picker, - GRPCServer: grpcSrv, - Cache: cache, + LocalPicker: conf.Picker, + GRPCServer: grpcSrv, + Cache: cache, + DataCenter: conf.DataCenter, }) checkErr(err, "while creating new gubernator instance") @@ -86,6 +89,23 @@ func main() { conf.K8PoolConf.OnUpdate = guber.SetPeers pool, err = gubernator.NewK8sPool(conf.K8PoolConf) checkErr(err, "while querying kubernetes API") + + } else if conf.MemberlistPoolConf.Enabled { + gubernatorPort, err := strconv.Atoi(strings.Split(conf.GRPCListenAddress, ":")[1]) + checkErr(err, "while converting gubernator port to int") + + // Register peer on memberlist + pool, err = gubernator.NewMemberlistPool(gubernator.MemberlistPoolConfig{ + AdvertiseAddress: conf.MemberlistPoolConf.AdvertiseAddress, + AdvertisePort: conf.MemberlistPoolConf.AdvertisePort, + KnownNodes: conf.MemberlistPoolConf.KnownNodes, + LoggerOutput: logrus.WithField("category", "memberlist").Writer(), + DataCenter: conf.DataCenter, + GubernatorPort: gubernatorPort, + OnUpdate: guber.SetPeers, + }) + checkErr(err, "while creating memberlist") + } else { // Register ourselves with other peers via ETCD etcdClient, err := etcdutil.NewClient(&conf.EtcdConf) diff --git a/config.go b/config.go index fc4e77f7..babe974c 100644 --- a/config.go +++ b/config.go @@ -45,9 +45,18 @@ type Config struct { // the contents of the cache when the gubernator instance is started and stopped Loader Loader - // (Optional) This is the peer picker algorithm the server will use decide which peer in the cluster - // will coordinate a rate limit - Picker PeerPicker + // (Optional) This is the peer picker algorithm the server will use decide which peer in the local cluster + // will own the rate limit + LocalPicker PeerPicker + + // (Optional) This is the peer picker algorithm the server will use when deciding which remote peer to forward + // rate limits too when a `Config.DataCenter` is set to something other than empty string. + RegionPicker RegionPeerPicker + + // (Optional) This is the name of our local data center. This value will be used by LocalPicker when + // deciding who we should immediately connect too for our local picker. Should remain empty if not + // using multi data center support. + DataCenter string } type BehaviorConfig struct { @@ -60,10 +69,17 @@ type BehaviorConfig struct { // How long a non-owning peer should wait before syncing hits to the owning peer GlobalSyncWait time.Duration - // How long we should wait for a global sync responses from peers + // How long we should wait for global sync responses from peers GlobalTimeout time.Duration // The max number of global updates we can batch into a single peer request GlobalBatchLimit int + + // How long the current region will collect request before pushing them to other regions + MultiRegionSyncWait time.Duration + // How long the current region will wait for responses from other regions + MultiRegionTimeout time.Duration + // The max number of requests the current region will collect + MultiRegionBatchLimit int } func (c *Config) SetDefaults() error { @@ -75,7 +91,12 @@ func (c *Config) SetDefaults() error { setter.SetDefault(&c.Behaviors.GlobalBatchLimit, maxBatchSize) setter.SetDefault(&c.Behaviors.GlobalSyncWait, time.Microsecond*500) - setter.SetDefault(&c.Picker, NewConsistantHash(nil)) + setter.SetDefault(&c.Behaviors.MultiRegionTimeout, time.Millisecond*500) + setter.SetDefault(&c.Behaviors.MultiRegionBatchLimit, maxBatchSize) + setter.SetDefault(&c.Behaviors.MultiRegionSyncWait, time.Second) + + setter.SetDefault(&c.LocalPicker, NewConsistantHash(nil)) + setter.SetDefault(&c.RegionPicker, NewRegionPicker(nil)) setter.SetDefault(&c.Cache, NewLRUCache(0)) if c.Behaviors.BatchLimit > maxBatchSize { diff --git a/docker-compose.yaml b/docker-compose.yaml index c5bb655e..04413e63 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -1,17 +1,5 @@ version: '3' services: - etcd: - image: quay.io/coreos/etcd:v3.2 - command: > - /usr/local/bin/etcd - -name etcd0 - -advertise-client-urls http://etcd:2379 - -listen-client-urls http://0.0.0.0:2379 - -initial-advertise-peer-urls http://0.0.0.0:2380 - -listen-peer-urls http://0.0.0.0:2380 - -initial-cluster-token etcd-cluster-1 - -initial-cluster etcd0=http://0.0.0.0:2380 - -initial-cluster-state new gubernator-1: image: thrawn01/gubernator:latest command: "/gubernator" @@ -20,17 +8,15 @@ services: - GUBER_GRPC_ADDRESS=0.0.0.0:81 # The address HTTP requests will listen on - GUBER_HTTP_ADDRESS=0.0.0.0:80 - # The address peers will connect too - # Should be the same as grpc-listen-address unless you are running behind - # a NAT or running in a docker container without host networking - - GUBER_ETCD_ADVERTISE_ADDRESS=gubernator-1:81 # Max size of the cache; The cache size will never grow beyond this size. - GUBER_CACHE_SIZE=50000 - # A Comma separate list of etcd nodes - - GUBER_ETCD_ENDPOINTS=etcd:2379 + # A comma separated list of known gubernator nodes + - GUBER_MEMBERLIST_KNOWN_NODES=gubernator-1,gubernator-2,gubernator-3,gubernator-4 + - GUBER_DATA_CENTER=us-east-1 ports: - "8081:81" - "8080:80" + gubernator-2: image: thrawn01/gubernator:latest command: "/gubernator" @@ -39,14 +25,45 @@ services: - GUBER_GRPC_ADDRESS=0.0.0.0:81 # The address HTTP requests will listen on - GUBER_HTTP_ADDRESS=0.0.0.0:80 - # The address peers will connect too - # Should be the same as grpc-listen-address unless you are running behind - # a NAT or running in a docker container without host networking - - GUBER_ETCD_ADVERTISE_ADDRESS=gubernator-2:81 # Max size of the cache; The cache size will never grow beyond this size. - GUBER_CACHE_SIZE=50000 - # A Comma separate list of etcd nodes - - GUBER_ETCD_ENDPOINTS=etcd:2379 + # A comma separated list of known gubernator nodes + - GUBER_MEMBERLIST_KNOWN_NODES=gubernator-1,gubernator-2,gubernator-3,gubernator-4 + - GUBER_DATA_CENTER=us-east-1 ports: - "8181:81" - "8180:80" + + gubernator-3: + image: thrawn01/gubernator:latest + command: "/gubernator" + environment: + # The address GRPC requests will listen on + - GUBER_GRPC_ADDRESS=0.0.0.0:81 + # The address HTTP requests will listen on + - GUBER_HTTP_ADDRESS=0.0.0.0:80 + # Max size of the cache; The cache size will never grow beyond this size. + - GUBER_CACHE_SIZE=50000 + # A comma separated list of known gubernator nodes + - GUBER_MEMBERLIST_KNOWN_NODES=gubernator-1,gubernator-2,gubernator-3,gubernator-4 + - GUBER_DATA_CENTER=us-west-2 + ports: + - "8281:81" + - "8280:80" + + gubernator-4: + image: thrawn01/gubernator:latest + command: "/gubernator" + environment: + # The address GRPC requests will listen on + - GUBER_GRPC_ADDRESS=0.0.0.0:81 + # The address HTTP requests will listen on + - GUBER_HTTP_ADDRESS=0.0.0.0:80 + # Max size of the cache; The cache size will never grow beyond this size. + - GUBER_CACHE_SIZE=50000 + # A Comma separate list of known gubernator nodes + - GUBER_MEMBERLIST_KNOWN_NODES=gubernator-1,gubernator-2,gubernator-3,gubernator-4 + - GUBER_DATA_CENTER=us-west-2 + ports: + - "8381:81" + - "8380:80" diff --git a/etcd.go b/etcd.go index 9b68f92c..ddf2b977 100644 --- a/etcd.go +++ b/etcd.go @@ -28,10 +28,22 @@ import ( ) type PeerInfo struct { + + // (Optional) The name of the data center this peer is in. Leave blank if not using multi data center support. + DataCenter string + + // (Required) The IP address of the peer which will field peer requests Address string + + // (Optional) Is true if PeerInfo is for this instance of gubernator IsOwner bool } +// Returns the hash key used to identify this peer in the Picker. +func (p PeerInfo) HashKey() string { + return p.Address +} + type UpdateFunc func([]PeerInfo) const ( @@ -290,7 +302,7 @@ func (e *EtcdPool) register(name string) error { if _, err := e.conf.Client.Revoke(ctx, lease.ID); err != nil { e.log.WithError(err). - Warn("during lease revoke ") + Warn("during lease revoke") } cancel() return false diff --git a/functional_test.go b/functional_test.go index 1e2f037e..62d5b249 100644 --- a/functional_test.go +++ b/functional_test.go @@ -31,7 +31,7 @@ import ( "github.com/stretchr/testify/require" ) -// Setup and shutdown the mailgun mock server for the entire test suite +// Setup and shutdown the mock gubernator cluster for the entire test suite func TestMain(m *testing.M) { if err := cluster.StartWith([]string{ "127.0.0.1:9990", @@ -49,7 +49,7 @@ func TestMain(m *testing.M) { } func TestOverTheLimit(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -80,6 +80,7 @@ func TestOverTheLimit(t *testing.T) { Duration: guber.Second, Limit: 2, Hits: 1, + Behavior: 0, }, }, }) @@ -95,7 +96,7 @@ func TestOverTheLimit(t *testing.T) { } func TestTokenBucket(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -147,7 +148,7 @@ func TestTokenBucket(t *testing.T) { } func TestLeakyBucket(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -208,7 +209,7 @@ func TestLeakyBucket(t *testing.T) { } func TestMissingFields(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -273,7 +274,7 @@ func TestMissingFields(t *testing.T) { func TestGlobalRateLimits(t *testing.T) { const clientInstance = 1 peer := cluster.PeerAt(clientInstance) - client, errs := guber.DialV1Server(peer) + client, errs := guber.DialV1Server(peer.Address) require.Nil(t, errs) ctx, cancel := context.WithTimeout(context.Background(), time.Second*3) @@ -324,7 +325,7 @@ func TestGlobalRateLimits(t *testing.T) { canonicalInstance := cluster.InstanceForHost(canonicalHost) // Inspect our metrics, ensure they collected the counts we expected during this test - instance := cluster.InstanceForHost(peer) + instance := cluster.InstanceForHost(peer.Address) metricCh := make(chan prometheus.Metric, 5) instance.Guber.Collect(metricCh) @@ -344,7 +345,7 @@ func TestGlobalRateLimits(t *testing.T) { } func TestChangeLimit(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -432,7 +433,7 @@ func TestChangeLimit(t *testing.T) { } func TestResetRemaining(t *testing.T) { - client, errs := guber.DialV1Server(cluster.GetPeer()) + client, errs := guber.DialV1Server(cluster.GetRandomPeer().Address) require.Nil(t, errs) tests := []struct { @@ -503,4 +504,68 @@ func TestResetRemaining(t *testing.T) { } } +func TestHealthCheck(t *testing.T) { + client, errs := guber.DialV1Server(cluster.InstanceAt(0).Address) + require.Nil(t, errs) + + // Check that the cluster is healthy to start with + healthResp, err := client.HealthCheck(context.Background(), &guber.HealthCheckReq{}) + require.Nil(t, err) + + assert.Equal(t, "healthy", healthResp.GetStatus()) + + // Create a global rate limit that will need to be sent to all peers in the cluster + _, err = client.GetRateLimits(context.Background(), &guber.GetRateLimitsReq{ + Requests: []*guber.RateLimitReq{ + { + Name: "test_health_check", + UniqueKey: "account:12345", + Algorithm: guber.Algorithm_TOKEN_BUCKET, + Behavior: guber.Behavior_GLOBAL, + Duration: guber.Second * 3, + Hits: 1, + Limit: 5, + }, + }, + }) + require.Nil(t, err) + + // Stop the rest of the cluster to ensure errors occur on our instance and + // collect addresses to restart the stopped instances after the test completes + var addresses []string + for i := 1; i < cluster.NumOfInstances(); i++ { + addresses = append(addresses, cluster.InstanceAt(i).Address) + cluster.StopInstanceAt(i) + } + time.Sleep(time.Second) + + // Hit the global rate limit again this time causing a connection error + _, err = client.GetRateLimits(context.Background(), &guber.GetRateLimitsReq{ + Requests: []*guber.RateLimitReq{ + { + Name: "test_health_check", + UniqueKey: "account:12345", + Algorithm: guber.Algorithm_TOKEN_BUCKET, + Behavior: guber.Behavior_GLOBAL, + Duration: guber.Second * 3, + Hits: 1, + Limit: 5, + }, + }, + }) + require.Nil(t, err) + + // Check the health again to get back the connection error + healthResp, err = client.HealthCheck(context.Background(), &guber.HealthCheckReq{}) + require.Nil(t, err) + + assert.Equal(t, "unhealthy", healthResp.GetStatus()) + assert.Contains(t, healthResp.GetMessage(), "connect: connection refused") + + // Restart stopped instances + for i := 1; i < cluster.NumOfInstances(); i++ { + cluster.StartInstance(addresses[i-1], cluster.GetDefaultConfig()) + } +} + // TODO: Add a test for sending no rate limits RateLimitReqList.RateLimits = nil diff --git a/global.go b/global.go index f07bb7dd..11eb99d1 100644 --- a/global.go +++ b/global.go @@ -129,11 +129,11 @@ func (gm *globalManager) sendHits(hits map[string]*RateLimitReq) { continue } - p, ok := peerRequests[peer.host] + p, ok := peerRequests[peer.info.Address] if ok { p.req.Requests = append(p.req.Requests, r) } else { - peerRequests[peer.host] = &pair{ + peerRequests[peer.info.Address] = &pair{ client: peer, req: GetPeerRateLimitsReq{Requests: []*RateLimitReq{r}}, } @@ -148,7 +148,7 @@ func (gm *globalManager) sendHits(hits map[string]*RateLimitReq) { if err != nil { gm.log.WithError(err). - Errorf("error sending global hits to '%s'", p.client.host) + Errorf("error sending global hits to '%s'", p.client.info.Address) continue } } @@ -218,7 +218,7 @@ func (gm *globalManager) updatePeers(updates map[string]*RateLimitReq) { for _, peer := range gm.instance.GetPeerList() { // Exclude ourselves from the update - if peer.isOwner { + if peer.info.IsOwner { continue } @@ -227,8 +227,9 @@ func (gm *globalManager) updatePeers(updates map[string]*RateLimitReq) { cancel() if err != nil { - if err != ErrClosing { - gm.log.WithError(err).Errorf("error sending global updates to '%s'", peer.host) + // Skip peers that are not in a ready state + if !IsNotReady(err) { + gm.log.WithError(err).Errorf("error sending global updates to '%s'", peer.info.Address) } continue } diff --git a/go.mod b/go.mod index d7c9f2e7..0a730953 100644 --- a/go.mod +++ b/go.mod @@ -3,8 +3,11 @@ module github.com/mailgun/gubernator require ( github.com/coreos/etcd v3.3.15+incompatible github.com/davecgh/go-spew v1.1.1 + github.com/fatih/structs v1.1.0 // indirect github.com/golang/protobuf v1.3.2 - github.com/grpc-ecosystem/grpc-gateway v1.10.0 + github.com/grpc-ecosystem/grpc-gateway v1.11.1 + github.com/hashicorp/memberlist v0.2.0 + github.com/mailgun/holster v3.0.0+incompatible github.com/mailgun/holster/v3 v3.8.1 github.com/pkg/errors v0.8.1 github.com/prometheus/client_golang v1.1.0 @@ -12,7 +15,6 @@ require ( github.com/segmentio/fasthash v1.0.2 github.com/sirupsen/logrus v1.4.2 github.com/stretchr/testify v1.4.0 - golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8 google.golang.org/grpc v1.23.0 k8s.io/api v0.0.0-20190620084959-7cf5895f2711 diff --git a/go.sum b/go.sum index 6f3fa1ff..9ff515d3 100644 --- a/go.sum +++ b/go.sum @@ -8,6 +8,8 @@ github.com/ahmetb/go-linq v3.0.0+incompatible h1:qQkjjOXKrKOTy83X8OpRmnKflXKQIL/ github.com/ahmetb/go-linq v3.0.0+incompatible/go.mod h1:PFffvbdbtw+QTB0WKRP0cNht7vnCfnGlEpak/DVg5cY= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da h1:8GUt8eRujhVEGZFFEjBj46YV4rDjvGrNxb0KMWYkL2I= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0= @@ -34,6 +36,8 @@ github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZm github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM= github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc= github.com/evanphx/json-patch v0.0.0-20190203023257-5858425f7550/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= +github.com/fatih/structs v1.1.0 h1:Q7juDM0QtcnhCpeyLGQKyg4TOIghuNXrkL32pHAUMxo= +github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -58,6 +62,7 @@ github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/google/btree v0.0.0-20160524151835-7d79101e329e/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= @@ -84,8 +89,24 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92Bcuy github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.10.0 h1:yqx/nTDLC6pVrQ8fTaCeeeMJNbmt7HglUpysQATYXV4= github.com/grpc-ecosystem/grpc-gateway v1.10.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/grpc-ecosystem/grpc-gateway v1.11.1 h1:/dBYI+n4xIL+Y9SKXQrjlKTmJJDwCSlNLRwZ5nBhIek= +github.com/grpc-ecosystem/grpc-gateway v1.11.1/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3 h1:zKjpN5BK/P5lMYrLmBHdBULWbJ0XpYR+7NGzqkZzoD4= +github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-sockaddr v1.0.0 h1:GeH6tui99pF4NJgfnhp+L6+FfobzVW3Ah46sLo0ICXs= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/memberlist v0.2.0 h1:WeeNspppWi5s1OFefTviPQueC/Bq8dONfvNjPhiEQKE= +github.com/hashicorp/memberlist v0.2.0/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/imdario/mergo v0.3.5/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= @@ -106,10 +127,14 @@ github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORN github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/mailgun/holster v3.0.0+incompatible h1:bpt8ZCwLBrzjqfBZ5mobNb2NjesNeDHmsOO++Ek9Swc= +github.com/mailgun/holster v3.0.0+incompatible/go.mod h1:crzolGx27RP/IBT/BnPQiYBB9igmAFHGRrz0zlMP0b0= github.com/mailgun/holster/v3 v3.8.1 h1:Z9D3F1ShnxGUlofougjSht08OpIiQKtryBjExB+uz9Q= github.com/mailgun/holster/v3 v3.8.1/go.mod h1:rNcFlhMTxFDa1dnQC4sUqI71IpAa9/aPeU6w8IGF3aQ= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/miekg/dns v1.1.26 h1:gPxPSwALAeHJSjarOs00QjVdV9QoBvc1D2ujQUr5BzU= +github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -120,6 +145,8 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/gomega v0.0.0-20190113212917-5533ce8a0da3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c h1:Lgl0gzECD8GnQ5QCWA8o6BtfL6mDH5rQgM4/fX3avOs= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= @@ -146,6 +173,8 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.3 h1:CTwfnzjQ+8dS6MhHHu4YswVAD99sL2wjPqP+VkURmKE= github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/segmentio/fasthash v1.0.2 h1:86fGDl2hB+iSHYlccB/FP9qRGvLNuH/fhEEFn6gnQUs= github.com/segmentio/fasthash v1.0.2/go.mod h1:waKX8l2N8yckOgmSsXJi7x1ZfdKZ4x7KRMzBtS3oedY= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= @@ -180,6 +209,8 @@ golang.org/x/crypto v0.0.0-20181025213731-e84da0312774 h1:a4tQYYYuK9QdeO/+kEvNYy golang.org/x/crypto v0.0.0-20181025213731-e84da0312774/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392 h1:ACG4HJsFiNMf47Y4PeRoebLNy/2lXT9EtprMuTFWt1M= +golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -191,8 +222,12 @@ golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20190206173232-65e2d4e15006 h1:bfLnR+k0tq5Lqt6dflRLcZiz6UaXCMt3vhYJ1l4FQ80= golang.org/x/net v0.0.0-20190206173232-65e2d4e15006/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 h1:dfGZHvZk057jK2MCeWus/TowKpJ8y4AmooUzdBSR9GU= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190923162816-aa69164e4478 h1:l5EDrHhldLYb3ZRHDUhXF7Om7MvYXnkV9/iQNo1lX6g= +golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a h1:tImsplftrFpALCYumobsd0K86vlAs/eXGFms2txfJfA= golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -200,6 +235,7 @@ golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20181108010431-42b317875d0f h1:Bl/8QSvNqXvPGPGXa2z5xUTmV7VDcZyvRZ+QQXkXTZQ= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33 h1:I6FyU15t786LL7oL/hn43zqTuEGr4PN7F4XJ1p4E3Y8= @@ -214,10 +250,15 @@ golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3 h1:4y9KwBHBgBNwDbtu44R5o1fdOCQUEXhbk/P4A9WmJq0= golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe h1:6fAMxZRR6sl1Uq8U61gxU+kPTs2tR8uOySCbBP7BN/M= +golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20181227161524-e6919f6577db h1:6/JqlYfC1CCaLnGceQTI+sDGhC9UBSPAsBqI0Gun6kU= golang.org/x/text v0.3.1-0.20181227161524-e6919f6577db/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/time v0.0.0-20161028155119-f51c12702a4d/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -226,6 +267,8 @@ golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0 h1:KxkO13IPW4Lslp2bz+KHP2E3gtFlrIGNThxkZQ3g+4c= @@ -235,6 +278,7 @@ google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoA google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0 h1:AzbTB6ux+okLTzP8Ru1Xs41C303zdcfEht7MQnYJt5A= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/gubernator.go b/gubernator.go index 67fd308c..dc4021d8 100644 --- a/gubernator.go +++ b/gubernator.go @@ -39,11 +39,12 @@ const ( var log *logrus.Entry type Instance struct { - health HealthCheckResp - global *globalManager - peerMutex sync.RWMutex - conf Config - isClosed bool + health HealthCheckResp + global *globalManager + mutliRegion *mutliRegionManager + peerMutex sync.RWMutex + conf Config + isClosed bool } func New(conf Config) (*Instance, error) { @@ -61,6 +62,7 @@ func New(conf Config) (*Instance, error) { } s.global = newGlobalManager(conf.Behaviors, &s) + s.mutliRegion = newMultiRegionManager(conf.Behaviors, &s) // Register our server with GRPC RegisterV1Server(conf.GRPCServer, &s) @@ -148,7 +150,7 @@ func (s *Instance) GetRateLimits(ctx context.Context, r *GetRateLimitsReq) (*Get getPeer: if attempts > 5 { inOut.Out = &RateLimitResp{ - Error: fmt.Sprintf("GetPeer() keeps returning peers that are in a closing state for '%s' - '%s'", globalKey, err), + Error: fmt.Sprintf("GetPeer() keeps returning peers that are not connected for '%s' - '%s'", globalKey, err), } out <- inOut return nil @@ -164,7 +166,7 @@ func (s *Instance) GetRateLimits(ctx context.Context, r *GetRateLimitsReq) (*Get } // If our server instance is the owner of this rate limit - if peer.isOwner { + if peer.info.IsOwner { // Apply our rate limit algorithm to the request inOut.Out, err = s.getRateLimit(inOut.In) if err != nil { @@ -180,7 +182,7 @@ func (s *Instance) GetRateLimits(ctx context.Context, r *GetRateLimitsReq) (*Get } // Inform the client of the owner key of the key - inOut.Out.Metadata = map[string]string{"owner": peer.host} + inOut.Out.Metadata = map[string]string{"owner": peer.info.Address} out <- inOut return nil @@ -189,7 +191,7 @@ func (s *Instance) GetRateLimits(ctx context.Context, r *GetRateLimitsReq) (*Get // Make an RPC call to the peer that owns this rate limit inOut.Out, err = peer.GetPeerRateLimit(ctx, inOut.In) if err != nil { - if err == ErrClosing { + if IsNotReady(err) { attempts++ goto getPeer } @@ -199,7 +201,7 @@ func (s *Instance) GetRateLimits(ctx context.Context, r *GetRateLimitsReq) (*Get } // Inform the client of the owner key of the key - inOut.Out.Metadata = map[string]string{"owner": peer.host} + inOut.Out.Metadata = map[string]string{"owner": peer.info.Address} } out <- inOut @@ -283,7 +285,41 @@ func (s *Instance) GetPeerRateLimits(ctx context.Context, r *GetPeerRateLimitsRe // HealthCheck Returns the health of our instance. func (s *Instance) HealthCheck(ctx context.Context, r *HealthCheckReq) (*HealthCheckResp, error) { + var errs []string + s.peerMutex.RLock() + + // Iterate through local peers and get their last errors + localPeers := s.conf.LocalPicker.Peers() + for _, peer := range localPeers { + lastErr := peer.GetLastErr() + + if lastErr != nil { + for _, err := range lastErr { + errs = append(errs, err) + } + } + } + + // Do the same for region peers + regionPeers := s.conf.RegionPicker.Peers() + for _, peer := range regionPeers { + lastErr := peer.GetLastErr() + + if lastErr != nil { + for _, err := range lastErr { + errs = append(errs, err) + } + } + } + + s.health.Status = Healthy + if len(errs) != 0 { + s.health.Status = UnHealthy + s.health.Message = strings.Join(errs, "|") + s.health.PeerCount = int32(s.conf.LocalPicker.Size()) + } + defer s.peerMutex.RUnlock() return &s.health, nil } @@ -296,6 +332,10 @@ func (s *Instance) getRateLimit(r *RateLimitReq) (*RateLimitResp, error) { s.global.QueueUpdate(r) } + if HasBehavior(r.Behavior, Behavior_MULTI_REGION) { + s.mutliRegion.QueueHits(r) + } + switch r.Algorithm { case Algorithm_TOKEN_BUCKET: return tokenBucket(s.conf.Store, s.conf.Cache, r) @@ -305,74 +345,72 @@ func (s *Instance) getRateLimit(r *RateLimitReq) (*RateLimitResp, error) { return nil, errors.Errorf("invalid rate limit algorithm '%d'", r.Algorithm) } -// SetPeers is called when the pool of peers changes -func (s *Instance) SetPeers(peers []PeerInfo) { - picker := s.conf.Picker.New() - var errs []string - - for _, peer := range peers { - peerInfo := s.conf.Picker.GetPeerByHost(peer.Address) - // If we don't have an existing connection, we need to open one. - if peerInfo == nil { - var err error - peerInfo, err = NewPeerClient(s.conf.Behaviors, peer.Address) - if err != nil { - errs = append(errs, - fmt.Sprintf("failed to connect to peer '%s'; consistent hash is incomplete", peer.Address)) - continue +// SetPeers is called by the implementor to indicate the pool of peers has changed +func (s *Instance) SetPeers(peerInfo []PeerInfo) { + localPicker := s.conf.LocalPicker.New() + regionPicker := s.conf.RegionPicker.New() + + for _, info := range peerInfo { + // Add peers that are not in our local DC to the RegionPicker + if info.DataCenter != s.conf.DataCenter { + peer := s.conf.RegionPicker.GetByPeerInfo(info) + // If we don't have an existing PeerClient create a new one + if peer == nil { + peer = NewPeerClient(s.conf.Behaviors, info) } - + regionPicker.Add(peer) + continue } - - // If this peer refers to this server instance - peerInfo.isOwner = peer.IsOwner - - picker.Add(peerInfo) + // If we don't have an existing PeerClient create a new one + peer := s.conf.LocalPicker.GetByPeerInfo(info) + if peer == nil { + peer = NewPeerClient(s.conf.Behaviors, info) + } + localPicker.Add(peer) } s.peerMutex.Lock() - - oldPicker := s.conf.Picker - - // Replace our current picker - s.conf.Picker = picker - - // Update our health status - s.health.Status = Healthy - if len(errs) != 0 { - s.health.Status = UnHealthy - s.health.Message = strings.Join(errs, "|") - } - s.health.PeerCount = int32(picker.Size()) + // Replace our current pickers + oldLocalPicker := s.conf.LocalPicker + oldRegionPicker := s.conf.RegionPicker + s.conf.LocalPicker = localPicker + s.conf.RegionPicker = regionPicker s.peerMutex.Unlock() - log.WithField("peers", peers).Info("Peers updated") + //TODO: This should include the regions peers? log.WithField("peers", peers).Info("Peers updated") // Shutdown any old peers we no longer need - wg := sync.WaitGroup{} ctx, cancel := context.WithTimeout(context.Background(), s.conf.Behaviors.BatchTimeout) defer cancel() var shutdownPeers []*PeerClient - for _, p := range oldPicker.Peers() { - // If this peerInfo is not found, we are no longer using this host and need to shut it down - if peerInfo := s.conf.Picker.GetPeerByHost(p.host); peerInfo == nil { - shutdownPeers = append(shutdownPeers, p) + for _, peer := range oldLocalPicker.Peers() { + if peerInfo := s.conf.LocalPicker.GetByPeerInfo(peer.info); peerInfo == nil { + shutdownPeers = append(shutdownPeers, peer) + } + } + + for _, regionPicker := range oldRegionPicker.Pickers() { + for _, peer := range regionPicker.Peers() { + if peerInfo := s.conf.RegionPicker.GetByPeerInfo(peer.info); peerInfo == nil { + shutdownPeers = append(shutdownPeers, peer) + } } } + var wg syncutil.WaitGroup for _, p := range shutdownPeers { - wg.Add(1) - go func(pc *PeerClient) { + wg.Run(func(obj interface{}) error { + pc := obj.(*PeerClient) err := pc.Shutdown(ctx) if err != nil { log.WithError(err).WithField("peer", pc).Error("while shutting down peer") } - wg.Done() - }(p) + return nil + }, p) } - wg.Wait() + if len(shutdownPeers) > 0 { log.WithField("peers", shutdownPeers).Info("Peers shutdown") } @@ -381,7 +419,7 @@ func (s *Instance) SetPeers(peers []PeerInfo) { // GetPeers returns a peer client for the hash key provided func (s *Instance) GetPeer(key string) (*PeerClient, error) { s.peerMutex.RLock() - peer, err := s.conf.Picker.Get(key) + peer, err := s.conf.LocalPicker.Get(key) if err != nil { s.peerMutex.RUnlock() return nil, err @@ -393,7 +431,13 @@ func (s *Instance) GetPeer(key string) (*PeerClient, error) { func (s *Instance) GetPeerList() []*PeerClient { s.peerMutex.RLock() defer s.peerMutex.RUnlock() - return s.conf.Picker.Peers() + return s.conf.LocalPicker.Peers() +} + +func (s *Instance) GetRegionPickers() map[string]PeerPicker { + s.peerMutex.RLock() + defer s.peerMutex.RUnlock() + return s.conf.RegionPicker.Pickers() } // Describe fetches prometheus metrics to be registered diff --git a/gubernator.pb.go b/gubernator.pb.go index f2257d6a..91092c96 100644 --- a/gubernator.pb.go +++ b/gubernator.pb.go @@ -1,36 +1,17 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // source: gubernator.proto -/* -Package gubernator is a generated protocol buffer package. - -It is generated from these files: - gubernator.proto - peers.proto - -It has these top-level messages: - GetRateLimitsReq - GetRateLimitsResp - RateLimitReq - RateLimitResp - HealthCheckReq - HealthCheckResp - GetPeerRateLimitsReq - GetPeerRateLimitsResp - UpdatePeerGlobalsReq - UpdatePeerGlobal - UpdatePeerGlobalsResp -*/ package gubernator -import proto "github.com/golang/protobuf/proto" -import fmt "fmt" -import math "math" -import _ "google.golang.org/genproto/googleapis/api/annotations" - import ( - context "golang.org/x/net/context" + context "context" + fmt "fmt" + proto "github.com/golang/protobuf/proto" + _ "google.golang.org/genproto/googleapis/api/annotations" grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + math "math" ) // Reference imports to suppress errors if they are not otherwise used. @@ -42,7 +23,7 @@ var _ = math.Inf // is compatible with the proto package it is being compiled against. // A compilation error at this line likely means your copy of the // proto package needs to be updated. -const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package +const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package type Algorithm int32 @@ -57,6 +38,7 @@ var Algorithm_name = map[int32]string{ 0: "TOKEN_BUCKET", 1: "LEAKY_BUCKET", } + var Algorithm_value = map[string]int32{ "TOKEN_BUCKET": 0, "LEAKY_BUCKET": 1, @@ -65,7 +47,10 @@ var Algorithm_value = map[string]int32{ func (x Algorithm) String() string { return proto.EnumName(Algorithm_name, int32(x)) } -func (Algorithm) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +func (Algorithm) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{0} +} // A set of int32 flags used to control the behavior of a rate limit in gubernator type Behavior int32 @@ -125,27 +110,35 @@ const ( // algorithm chosen. For instance, if used with `TOKEN_BUCKET` it will immediately expire the // cache value. For `LEAKY_BUCKET` it sets the `Remaining` to `Limit`. Behavior_RESET_REMAINING Behavior = 8 + // Pushes rate limits to other regions + Behavior_MULTI_REGION Behavior = 16 ) var Behavior_name = map[int32]string{ - 0: "BATCHING", - 1: "NO_BATCHING", - 2: "GLOBAL", - 4: "DURATION_IS_GREGORIAN", - 8: "RESET_REMAINING", + 0: "BATCHING", + 1: "NO_BATCHING", + 2: "GLOBAL", + 4: "DURATION_IS_GREGORIAN", + 8: "RESET_REMAINING", + 16: "MULTI_REGION", } + var Behavior_value = map[string]int32{ "BATCHING": 0, "NO_BATCHING": 1, "GLOBAL": 2, "DURATION_IS_GREGORIAN": 4, "RESET_REMAINING": 8, + "MULTI_REGION": 16, } func (x Behavior) String() string { return proto.EnumName(Behavior_name, int32(x)) } -func (Behavior) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +func (Behavior) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{1} +} type Status int32 @@ -158,6 +151,7 @@ var Status_name = map[int32]string{ 0: "UNDER_LIMIT", 1: "OVER_LIMIT", } + var Status_value = map[string]int32{ "UNDER_LIMIT": 0, "OVER_LIMIT": 1, @@ -166,17 +160,43 @@ var Status_value = map[string]int32{ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } -func (Status) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +func (Status) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{2} +} // Must specify at least one Request type GetRateLimitsReq struct { - Requests []*RateLimitReq `protobuf:"bytes,1,rep,name=requests" json:"requests,omitempty"` + Requests []*RateLimitReq `protobuf:"bytes,1,rep,name=requests,proto3" json:"requests,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GetRateLimitsReq) Reset() { *m = GetRateLimitsReq{} } -func (m *GetRateLimitsReq) String() string { return proto.CompactTextString(m) } -func (*GetRateLimitsReq) ProtoMessage() {} -func (*GetRateLimitsReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } +func (m *GetRateLimitsReq) Reset() { *m = GetRateLimitsReq{} } +func (m *GetRateLimitsReq) String() string { return proto.CompactTextString(m) } +func (*GetRateLimitsReq) ProtoMessage() {} +func (*GetRateLimitsReq) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{0} +} + +func (m *GetRateLimitsReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetRateLimitsReq.Unmarshal(m, b) +} +func (m *GetRateLimitsReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetRateLimitsReq.Marshal(b, m, deterministic) +} +func (m *GetRateLimitsReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetRateLimitsReq.Merge(m, src) +} +func (m *GetRateLimitsReq) XXX_Size() int { + return xxx_messageInfo_GetRateLimitsReq.Size(m) +} +func (m *GetRateLimitsReq) XXX_DiscardUnknown() { + xxx_messageInfo_GetRateLimitsReq.DiscardUnknown(m) +} + +var xxx_messageInfo_GetRateLimitsReq proto.InternalMessageInfo func (m *GetRateLimitsReq) GetRequests() []*RateLimitReq { if m != nil { @@ -187,13 +207,36 @@ func (m *GetRateLimitsReq) GetRequests() []*RateLimitReq { // RateLimits returned are in the same order as the Requests type GetRateLimitsResp struct { - Responses []*RateLimitResp `protobuf:"bytes,1,rep,name=responses" json:"responses,omitempty"` + Responses []*RateLimitResp `protobuf:"bytes,1,rep,name=responses,proto3" json:"responses,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetRateLimitsResp) Reset() { *m = GetRateLimitsResp{} } +func (m *GetRateLimitsResp) String() string { return proto.CompactTextString(m) } +func (*GetRateLimitsResp) ProtoMessage() {} +func (*GetRateLimitsResp) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{1} } -func (m *GetRateLimitsResp) Reset() { *m = GetRateLimitsResp{} } -func (m *GetRateLimitsResp) String() string { return proto.CompactTextString(m) } -func (*GetRateLimitsResp) ProtoMessage() {} -func (*GetRateLimitsResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +func (m *GetRateLimitsResp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetRateLimitsResp.Unmarshal(m, b) +} +func (m *GetRateLimitsResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetRateLimitsResp.Marshal(b, m, deterministic) +} +func (m *GetRateLimitsResp) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetRateLimitsResp.Merge(m, src) +} +func (m *GetRateLimitsResp) XXX_Size() int { + return xxx_messageInfo_GetRateLimitsResp.Size(m) +} +func (m *GetRateLimitsResp) XXX_DiscardUnknown() { + xxx_messageInfo_GetRateLimitsResp.DiscardUnknown(m) +} + +var xxx_messageInfo_GetRateLimitsResp proto.InternalMessageInfo func (m *GetRateLimitsResp) GetResponses() []*RateLimitResp { if m != nil { @@ -204,30 +247,53 @@ func (m *GetRateLimitsResp) GetResponses() []*RateLimitResp { type RateLimitReq struct { // The name of the rate limit IE: 'requests_per_second', 'gets_per_minute` - Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // Uniquely identifies this rate limit IE: 'ip:10.2.10.7' or 'account:123445' - UniqueKey string `protobuf:"bytes,2,opt,name=unique_key,json=uniqueKey" json:"unique_key,omitempty"` + UniqueKey string `protobuf:"bytes,2,opt,name=unique_key,json=uniqueKey,proto3" json:"unique_key,omitempty"` // Rate limit requests optionally specify the number of hits a request adds to the matched limit. If Hit // is zero, the request returns the current limit, but does not increment the hit count. - Hits int64 `protobuf:"varint,3,opt,name=hits" json:"hits,omitempty"` + Hits int64 `protobuf:"varint,3,opt,name=hits,proto3" json:"hits,omitempty"` // The number of requests that can occur for the duration of the rate limit - Limit int64 `protobuf:"varint,4,opt,name=limit" json:"limit,omitempty"` + Limit int64 `protobuf:"varint,4,opt,name=limit,proto3" json:"limit,omitempty"` // The duration of the rate limit in milliseconds // Second = 1000 Milliseconds // Minute = 60000 Milliseconds // Hour = 3600000 Milliseconds - Duration int64 `protobuf:"varint,5,opt,name=duration" json:"duration,omitempty"` + Duration int64 `protobuf:"varint,5,opt,name=duration,proto3" json:"duration,omitempty"` // The algorithm used to calculate the rate limit. The algorithm may change on // subsequent requests, when this occurs any previous rate limit hit counts are reset. - Algorithm Algorithm `protobuf:"varint,6,opt,name=algorithm,enum=pb.gubernator.Algorithm" json:"algorithm,omitempty"` + Algorithm Algorithm `protobuf:"varint,6,opt,name=algorithm,proto3,enum=pb.gubernator.Algorithm" json:"algorithm,omitempty"` // Behavior is a set of int32 flags that control the behavior of the rate limit in gubernator - Behavior Behavior `protobuf:"varint,7,opt,name=behavior,enum=pb.gubernator.Behavior" json:"behavior,omitempty"` + Behavior Behavior `protobuf:"varint,7,opt,name=behavior,proto3,enum=pb.gubernator.Behavior" json:"behavior,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RateLimitReq) Reset() { *m = RateLimitReq{} } +func (m *RateLimitReq) String() string { return proto.CompactTextString(m) } +func (*RateLimitReq) ProtoMessage() {} +func (*RateLimitReq) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{2} +} + +func (m *RateLimitReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RateLimitReq.Unmarshal(m, b) +} +func (m *RateLimitReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RateLimitReq.Marshal(b, m, deterministic) +} +func (m *RateLimitReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_RateLimitReq.Merge(m, src) +} +func (m *RateLimitReq) XXX_Size() int { + return xxx_messageInfo_RateLimitReq.Size(m) +} +func (m *RateLimitReq) XXX_DiscardUnknown() { + xxx_messageInfo_RateLimitReq.DiscardUnknown(m) } -func (m *RateLimitReq) Reset() { *m = RateLimitReq{} } -func (m *RateLimitReq) String() string { return proto.CompactTextString(m) } -func (*RateLimitReq) ProtoMessage() {} -func (*RateLimitReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +var xxx_messageInfo_RateLimitReq proto.InternalMessageInfo func (m *RateLimitReq) GetName() string { if m != nil { @@ -280,23 +346,46 @@ func (m *RateLimitReq) GetBehavior() Behavior { type RateLimitResp struct { // The status of the rate limit. - Status Status `protobuf:"varint,1,opt,name=status,enum=pb.gubernator.Status" json:"status,omitempty"` + Status Status `protobuf:"varint,1,opt,name=status,proto3,enum=pb.gubernator.Status" json:"status,omitempty"` // The currently configured request limit (Identical to RateLimitRequest.rate_limit_config.limit). - Limit int64 `protobuf:"varint,2,opt,name=limit" json:"limit,omitempty"` + Limit int64 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` // This is the number of requests remaining before the limit is hit. - Remaining int64 `protobuf:"varint,3,opt,name=remaining" json:"remaining,omitempty"` + Remaining int64 `protobuf:"varint,3,opt,name=remaining,proto3" json:"remaining,omitempty"` // This is the time when the rate limit span will be reset, provided as a unix timestamp in milliseconds. - ResetTime int64 `protobuf:"varint,4,opt,name=reset_time,json=resetTime" json:"reset_time,omitempty"` + ResetTime int64 `protobuf:"varint,4,opt,name=reset_time,json=resetTime,proto3" json:"reset_time,omitempty"` // Contains the error; If set all other values should be ignored - Error string `protobuf:"bytes,5,opt,name=error" json:"error,omitempty"` + Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` // This is additional metadata that a client might find useful. (IE: Additional headers, corrdinator ownership, etc..) - Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RateLimitResp) Reset() { *m = RateLimitResp{} } -func (m *RateLimitResp) String() string { return proto.CompactTextString(m) } -func (*RateLimitResp) ProtoMessage() {} -func (*RateLimitResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } +func (m *RateLimitResp) Reset() { *m = RateLimitResp{} } +func (m *RateLimitResp) String() string { return proto.CompactTextString(m) } +func (*RateLimitResp) ProtoMessage() {} +func (*RateLimitResp) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{3} +} + +func (m *RateLimitResp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RateLimitResp.Unmarshal(m, b) +} +func (m *RateLimitResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RateLimitResp.Marshal(b, m, deterministic) +} +func (m *RateLimitResp) XXX_Merge(src proto.Message) { + xxx_messageInfo_RateLimitResp.Merge(m, src) +} +func (m *RateLimitResp) XXX_Size() int { + return xxx_messageInfo_RateLimitResp.Size(m) +} +func (m *RateLimitResp) XXX_DiscardUnknown() { + xxx_messageInfo_RateLimitResp.DiscardUnknown(m) +} + +var xxx_messageInfo_RateLimitResp proto.InternalMessageInfo func (m *RateLimitResp) GetStatus() Status { if m != nil { @@ -341,26 +430,72 @@ func (m *RateLimitResp) GetMetadata() map[string]string { } type HealthCheckReq struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HealthCheckReq) Reset() { *m = HealthCheckReq{} } +func (m *HealthCheckReq) String() string { return proto.CompactTextString(m) } +func (*HealthCheckReq) ProtoMessage() {} +func (*HealthCheckReq) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{4} +} + +func (m *HealthCheckReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HealthCheckReq.Unmarshal(m, b) +} +func (m *HealthCheckReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HealthCheckReq.Marshal(b, m, deterministic) +} +func (m *HealthCheckReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthCheckReq.Merge(m, src) +} +func (m *HealthCheckReq) XXX_Size() int { + return xxx_messageInfo_HealthCheckReq.Size(m) +} +func (m *HealthCheckReq) XXX_DiscardUnknown() { + xxx_messageInfo_HealthCheckReq.DiscardUnknown(m) } -func (m *HealthCheckReq) Reset() { *m = HealthCheckReq{} } -func (m *HealthCheckReq) String() string { return proto.CompactTextString(m) } -func (*HealthCheckReq) ProtoMessage() {} -func (*HealthCheckReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } +var xxx_messageInfo_HealthCheckReq proto.InternalMessageInfo type HealthCheckResp struct { // Valid entries are 'healthy' or 'unhealthy' - Status string `protobuf:"bytes,1,opt,name=status" json:"status,omitempty"` + Status string `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` // If 'unhealthy', message indicates the problem - Message string `protobuf:"bytes,2,opt,name=message" json:"message,omitempty"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // The number of peers we know about - PeerCount int32 `protobuf:"varint,3,opt,name=peer_count,json=peerCount" json:"peer_count,omitempty"` + PeerCount int32 `protobuf:"varint,3,opt,name=peer_count,json=peerCount,proto3" json:"peer_count,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HealthCheckResp) Reset() { *m = HealthCheckResp{} } -func (m *HealthCheckResp) String() string { return proto.CompactTextString(m) } -func (*HealthCheckResp) ProtoMessage() {} -func (*HealthCheckResp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +func (m *HealthCheckResp) Reset() { *m = HealthCheckResp{} } +func (m *HealthCheckResp) String() string { return proto.CompactTextString(m) } +func (*HealthCheckResp) ProtoMessage() {} +func (*HealthCheckResp) Descriptor() ([]byte, []int) { + return fileDescriptor_079522a6f9470154, []int{5} +} + +func (m *HealthCheckResp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HealthCheckResp.Unmarshal(m, b) +} +func (m *HealthCheckResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HealthCheckResp.Marshal(b, m, deterministic) +} +func (m *HealthCheckResp) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthCheckResp.Merge(m, src) +} +func (m *HealthCheckResp) XXX_Size() int { + return xxx_messageInfo_HealthCheckResp.Size(m) +} +func (m *HealthCheckResp) XXX_DiscardUnknown() { + xxx_messageInfo_HealthCheckResp.DiscardUnknown(m) +} + +var xxx_messageInfo_HealthCheckResp proto.InternalMessageInfo func (m *HealthCheckResp) GetStatus() string { if m != nil { @@ -384,15 +519,67 @@ func (m *HealthCheckResp) GetPeerCount() int32 { } func init() { + proto.RegisterEnum("pb.gubernator.Algorithm", Algorithm_name, Algorithm_value) + proto.RegisterEnum("pb.gubernator.Behavior", Behavior_name, Behavior_value) + proto.RegisterEnum("pb.gubernator.Status", Status_name, Status_value) proto.RegisterType((*GetRateLimitsReq)(nil), "pb.gubernator.GetRateLimitsReq") proto.RegisterType((*GetRateLimitsResp)(nil), "pb.gubernator.GetRateLimitsResp") proto.RegisterType((*RateLimitReq)(nil), "pb.gubernator.RateLimitReq") proto.RegisterType((*RateLimitResp)(nil), "pb.gubernator.RateLimitResp") + proto.RegisterMapType((map[string]string)(nil), "pb.gubernator.RateLimitResp.MetadataEntry") proto.RegisterType((*HealthCheckReq)(nil), "pb.gubernator.HealthCheckReq") proto.RegisterType((*HealthCheckResp)(nil), "pb.gubernator.HealthCheckResp") - proto.RegisterEnum("pb.gubernator.Algorithm", Algorithm_name, Algorithm_value) - proto.RegisterEnum("pb.gubernator.Behavior", Behavior_name, Behavior_value) - proto.RegisterEnum("pb.gubernator.Status", Status_name, Status_value) +} + +func init() { proto.RegisterFile("gubernator.proto", fileDescriptor_079522a6f9470154) } + +var fileDescriptor_079522a6f9470154 = []byte{ + // 712 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0xdd, 0x6e, 0xda, 0x48, + 0x14, 0x8e, 0x21, 0x21, 0xf8, 0x24, 0x80, 0x33, 0xbb, 0x49, 0xbc, 0x2c, 0xd9, 0x45, 0xbe, 0x62, + 0x91, 0x16, 0x14, 0x22, 0xed, 0xae, 0xb2, 0x57, 0x40, 0x5c, 0x62, 0x01, 0xb6, 0x34, 0x81, 0x48, + 0xed, 0x8d, 0x35, 0x24, 0x23, 0xb0, 0x82, 0x7f, 0xf0, 0x8c, 0x53, 0xe5, 0xae, 0xea, 0x2b, 0xf4, + 0x35, 0xfa, 0x36, 0xbd, 0xee, 0x5d, 0x1f, 0xa1, 0x0f, 0x50, 0x8d, 0x01, 0x83, 0x91, 0x9a, 0xbb, + 0x39, 0xdf, 0xf9, 0xce, 0x77, 0x3c, 0xdf, 0x07, 0x03, 0xca, 0x34, 0x9a, 0xd0, 0xd0, 0x23, 0xdc, + 0x0f, 0x1b, 0x41, 0xe8, 0x73, 0x1f, 0x15, 0x82, 0x49, 0x63, 0x03, 0x96, 0x2b, 0x53, 0xdf, 0x9f, + 0xce, 0x69, 0x93, 0x04, 0x4e, 0x93, 0x78, 0x9e, 0xcf, 0x09, 0x77, 0x7c, 0x8f, 0x2d, 0xc9, 0x5a, + 0x1f, 0x94, 0x1e, 0xe5, 0x98, 0x70, 0x3a, 0x70, 0x5c, 0x87, 0x33, 0x4c, 0x17, 0xe8, 0x5f, 0xc8, + 0x87, 0x74, 0x11, 0x51, 0xc6, 0x99, 0x2a, 0x55, 0xb3, 0xb5, 0xa3, 0xd6, 0xef, 0x8d, 0x94, 0x66, + 0x23, 0xe1, 0x63, 0xba, 0xc0, 0x09, 0x59, 0xb3, 0xe0, 0x64, 0x47, 0x8c, 0x05, 0xe8, 0x1a, 0xe4, + 0x90, 0xb2, 0xc0, 0xf7, 0x18, 0x5d, 0xcb, 0x55, 0x7e, 0x2e, 0xc7, 0x02, 0xbc, 0xa1, 0x6b, 0xdf, + 0x25, 0x38, 0xde, 0xde, 0x85, 0x10, 0xec, 0x7b, 0xc4, 0xa5, 0xaa, 0x54, 0x95, 0x6a, 0x32, 0x8e, + 0xcf, 0xe8, 0x02, 0x20, 0xf2, 0x9c, 0x45, 0x44, 0xed, 0x27, 0xfa, 0xa2, 0x66, 0xe2, 0x8e, 0xbc, + 0x44, 0xfa, 0xf4, 0x45, 0x8c, 0xcc, 0x1c, 0xce, 0xd4, 0x6c, 0x55, 0xaa, 0x65, 0x71, 0x7c, 0x46, + 0xbf, 0xc2, 0xc1, 0x5c, 0x48, 0xaa, 0xfb, 0x31, 0xb8, 0x2c, 0x50, 0x19, 0xf2, 0x8f, 0x51, 0x18, + 0xdb, 0xa3, 0x1e, 0xc4, 0x8d, 0xa4, 0x46, 0xff, 0x80, 0x4c, 0xe6, 0x53, 0x3f, 0x74, 0xf8, 0xcc, + 0x55, 0x73, 0x55, 0xa9, 0x56, 0x6c, 0xa9, 0x3b, 0xb7, 0x68, 0xaf, 0xfb, 0x78, 0x43, 0x45, 0x57, + 0x90, 0x9f, 0xd0, 0x19, 0x79, 0x76, 0xfc, 0x50, 0x3d, 0x8c, 0xc7, 0xce, 0x77, 0xc6, 0x3a, 0xab, + 0x36, 0x4e, 0x88, 0xda, 0xe7, 0x0c, 0x14, 0x52, 0x9e, 0xa0, 0xbf, 0x21, 0xc7, 0x38, 0xe1, 0x11, + 0x8b, 0x6f, 0x5e, 0x6c, 0x9d, 0xee, 0x88, 0xdc, 0xc5, 0x4d, 0xbc, 0x22, 0x6d, 0xee, 0x97, 0xd9, + 0xbe, 0x5f, 0x45, 0x24, 0xe1, 0x12, 0xc7, 0x73, 0xbc, 0xe9, 0xca, 0x8e, 0x0d, 0x20, 0x6c, 0x0c, + 0x29, 0xa3, 0xdc, 0xe6, 0x8e, 0x4b, 0x57, 0xc6, 0xc8, 0x31, 0x32, 0x72, 0x5c, 0x2a, 0x24, 0x69, + 0x18, 0xfa, 0x61, 0xec, 0x8c, 0x8c, 0x97, 0x05, 0x7a, 0x03, 0x79, 0x97, 0x72, 0xf2, 0x48, 0x38, + 0x51, 0x73, 0x71, 0xb6, 0xf5, 0xd7, 0xb2, 0x6d, 0x0c, 0x57, 0x64, 0xdd, 0xe3, 0xe1, 0x0b, 0x4e, + 0x66, 0xcb, 0xff, 0x43, 0x21, 0xd5, 0x42, 0x0a, 0x64, 0x45, 0x9a, 0xcb, 0x9c, 0xc5, 0x51, 0x7c, + 0xc0, 0x33, 0x99, 0x47, 0x74, 0x95, 0xf0, 0xb2, 0xb8, 0xce, 0xfc, 0x27, 0x69, 0x0a, 0x14, 0x6f, + 0x29, 0x99, 0xf3, 0x59, 0x77, 0x46, 0x1f, 0x9e, 0x30, 0x5d, 0x68, 0x13, 0x28, 0xa5, 0x10, 0x16, + 0xa0, 0xb3, 0x94, 0x83, 0x72, 0x62, 0x95, 0x0a, 0x87, 0x2e, 0x65, 0x8c, 0x4c, 0xd7, 0xc2, 0xeb, + 0x52, 0x18, 0x12, 0x50, 0x1a, 0xda, 0x0f, 0x7e, 0xe4, 0xf1, 0xd8, 0xaf, 0x03, 0x2c, 0x0b, 0xa4, + 0x2b, 0x80, 0x7a, 0x13, 0xe4, 0x24, 0x71, 0xa4, 0xc0, 0xf1, 0xc8, 0xea, 0xeb, 0xa6, 0xdd, 0x19, + 0x77, 0xfb, 0xfa, 0x48, 0xd9, 0x13, 0xc8, 0x40, 0x6f, 0xf7, 0xdf, 0xae, 0x11, 0xa9, 0xfe, 0x1e, + 0xf2, 0xeb, 0xac, 0xd1, 0x31, 0xe4, 0x3b, 0xed, 0x51, 0xf7, 0xd6, 0x30, 0x7b, 0xca, 0x1e, 0x2a, + 0xc1, 0x91, 0x69, 0xd9, 0x09, 0x20, 0x21, 0x80, 0x5c, 0x6f, 0x60, 0x75, 0xda, 0x03, 0x25, 0x83, + 0x7e, 0x83, 0xd3, 0x9b, 0x31, 0x6e, 0x8f, 0x0c, 0xcb, 0xb4, 0x8d, 0x3b, 0xbb, 0x87, 0xf5, 0x9e, + 0x85, 0x8d, 0xb6, 0xa9, 0xec, 0xa3, 0x5f, 0xa0, 0x84, 0xf5, 0x3b, 0x7d, 0x64, 0x63, 0x7d, 0xd8, + 0x36, 0x4c, 0x31, 0x9b, 0x17, 0x8b, 0x87, 0xe3, 0xc1, 0xc8, 0xb0, 0xb1, 0xde, 0x33, 0x2c, 0x53, + 0x51, 0xea, 0x7f, 0x41, 0x6e, 0xf9, 0xfb, 0x10, 0x8b, 0xc6, 0xe6, 0x8d, 0x8e, 0xed, 0x81, 0x31, + 0x34, 0xc4, 0x57, 0x16, 0x01, 0xac, 0xfb, 0xa4, 0x96, 0x5a, 0x5f, 0x25, 0xc8, 0xdc, 0x5f, 0xa2, + 0x00, 0x0a, 0xa9, 0x3f, 0x32, 0xfa, 0x73, 0x27, 0xd5, 0xdd, 0x37, 0xa3, 0x5c, 0x7d, 0x9d, 0xc0, + 0x02, 0xad, 0xf2, 0xf1, 0xcb, 0xb7, 0x4f, 0x99, 0x33, 0xed, 0xa4, 0xf9, 0x7c, 0xd9, 0x4c, 0xb5, + 0xaf, 0xa5, 0x3a, 0xa2, 0x70, 0xb4, 0x95, 0x18, 0xba, 0xd8, 0x91, 0x4b, 0xe7, 0x5b, 0xfe, 0xe3, + 0xb5, 0x36, 0x0b, 0xb4, 0xf3, 0x78, 0xd7, 0x09, 0x2a, 0x89, 0x5d, 0x5b, 0xcd, 0x4e, 0xe9, 0x1d, + 0x6c, 0xc6, 0x3e, 0x48, 0xd2, 0x24, 0x17, 0x3f, 0x83, 0x57, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, + 0x67, 0x9a, 0x70, 0x17, 0x47, 0x05, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -403,8 +590,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for V1 service - +// V1Client is the client API for V1 service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type V1Client interface { // Given a list of rate limit requests, return the rate limits of each. GetRateLimits(ctx context.Context, in *GetRateLimitsReq, opts ...grpc.CallOption) (*GetRateLimitsResp, error) @@ -423,7 +611,7 @@ func NewV1Client(cc *grpc.ClientConn) V1Client { func (c *v1Client) GetRateLimits(ctx context.Context, in *GetRateLimitsReq, opts ...grpc.CallOption) (*GetRateLimitsResp, error) { out := new(GetRateLimitsResp) - err := grpc.Invoke(ctx, "/pb.gubernator.V1/GetRateLimits", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/pb.gubernator.V1/GetRateLimits", in, out, opts...) if err != nil { return nil, err } @@ -432,15 +620,14 @@ func (c *v1Client) GetRateLimits(ctx context.Context, in *GetRateLimitsReq, opts func (c *v1Client) HealthCheck(ctx context.Context, in *HealthCheckReq, opts ...grpc.CallOption) (*HealthCheckResp, error) { out := new(HealthCheckResp) - err := grpc.Invoke(ctx, "/pb.gubernator.V1/HealthCheck", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/pb.gubernator.V1/HealthCheck", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for V1 service - +// V1Server is the server API for V1 service. type V1Server interface { // Given a list of rate limit requests, return the rate limits of each. GetRateLimits(context.Context, *GetRateLimitsReq) (*GetRateLimitsResp, error) @@ -449,6 +636,17 @@ type V1Server interface { HealthCheck(context.Context, *HealthCheckReq) (*HealthCheckResp, error) } +// UnimplementedV1Server can be embedded to have forward compatible implementations. +type UnimplementedV1Server struct { +} + +func (*UnimplementedV1Server) GetRateLimits(ctx context.Context, req *GetRateLimitsReq) (*GetRateLimitsResp, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRateLimits not implemented") +} +func (*UnimplementedV1Server) HealthCheck(ctx context.Context, req *HealthCheckReq) (*HealthCheckResp, error) { + return nil, status.Errorf(codes.Unimplemented, "method HealthCheck not implemented") +} + func RegisterV1Server(s *grpc.Server, srv V1Server) { s.RegisterService(&_V1_serviceDesc, srv) } @@ -505,53 +703,3 @@ var _V1_serviceDesc = grpc.ServiceDesc{ Streams: []grpc.StreamDesc{}, Metadata: "gubernator.proto", } - -func init() { proto.RegisterFile("gubernator.proto", fileDescriptor0) } - -var fileDescriptor0 = []byte{ - // 697 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0xdd, 0x6e, 0xda, 0x48, - 0x18, 0x8d, 0x21, 0x21, 0xf8, 0x4b, 0x00, 0x67, 0x76, 0x93, 0x78, 0x59, 0xb2, 0x8b, 0x7c, 0xc5, - 0x22, 0x2d, 0x28, 0x44, 0xda, 0xad, 0xd2, 0x2b, 0x20, 0x2e, 0x41, 0x10, 0x90, 0x26, 0x24, 0x52, - 0x7b, 0x63, 0x0d, 0xc9, 0x27, 0xb0, 0x82, 0x7f, 0xf0, 0x8c, 0x23, 0xe5, 0xae, 0xea, 0x2b, 0xf4, - 0x35, 0xfa, 0x36, 0xbd, 0xee, 0x5d, 0x1f, 0xa1, 0x0f, 0x50, 0x8d, 0x01, 0x83, 0x91, 0x9a, 0x3b, - 0x7f, 0xe7, 0x9c, 0xef, 0x8c, 0xe7, 0x1c, 0x30, 0x68, 0x93, 0x70, 0x8c, 0x81, 0xcb, 0x84, 0x17, - 0xd4, 0xfc, 0xc0, 0x13, 0x1e, 0xc9, 0xf9, 0xe3, 0xda, 0x1a, 0x2c, 0x96, 0x26, 0x9e, 0x37, 0x99, - 0x61, 0x9d, 0xf9, 0x76, 0x9d, 0xb9, 0xae, 0x27, 0x98, 0xb0, 0x3d, 0x97, 0x2f, 0xc4, 0x46, 0x0f, - 0xb4, 0x0e, 0x0a, 0xca, 0x04, 0xf6, 0x6d, 0xc7, 0x16, 0x9c, 0xe2, 0x9c, 0xfc, 0x0f, 0xd9, 0x00, - 0xe7, 0x21, 0x72, 0xc1, 0x75, 0xa5, 0x9c, 0xae, 0x1c, 0x34, 0xfe, 0xac, 0x25, 0x3c, 0x6b, 0xb1, - 0x9e, 0xe2, 0x9c, 0xc6, 0x62, 0x63, 0x08, 0x47, 0x5b, 0x66, 0xdc, 0x27, 0x97, 0xa0, 0x06, 0xc8, - 0x7d, 0xcf, 0xe5, 0xb8, 0xb2, 0x2b, 0xfd, 0xda, 0x8e, 0xfb, 0x74, 0x2d, 0x37, 0x7e, 0x28, 0x70, - 0xb8, 0x79, 0x16, 0x21, 0xb0, 0xeb, 0x32, 0x07, 0x75, 0xa5, 0xac, 0x54, 0x54, 0x1a, 0x3d, 0x93, - 0x33, 0x80, 0xd0, 0xb5, 0xe7, 0x21, 0x5a, 0x4f, 0xf8, 0xa2, 0xa7, 0x22, 0x46, 0x5d, 0x20, 0x3d, - 0x7c, 0x91, 0x2b, 0x53, 0x5b, 0x70, 0x3d, 0x5d, 0x56, 0x2a, 0x69, 0x1a, 0x3d, 0x93, 0xdf, 0x61, - 0x6f, 0x26, 0x2d, 0xf5, 0xdd, 0x08, 0x5c, 0x0c, 0xa4, 0x08, 0xd9, 0xc7, 0x30, 0x88, 0xe2, 0xd1, - 0xf7, 0x22, 0x22, 0x9e, 0xc9, 0x7f, 0xa0, 0xb2, 0xd9, 0xc4, 0x0b, 0x6c, 0x31, 0x75, 0xf4, 0x4c, - 0x59, 0xa9, 0xe4, 0x1b, 0xfa, 0xd6, 0x2d, 0x9a, 0x2b, 0x9e, 0xae, 0xa5, 0xe4, 0x02, 0xb2, 0x63, - 0x9c, 0xb2, 0x67, 0xdb, 0x0b, 0xf4, 0xfd, 0x68, 0xed, 0x74, 0x6b, 0xad, 0xb5, 0xa4, 0x69, 0x2c, - 0x34, 0xbe, 0xa4, 0x20, 0x97, 0xc8, 0x84, 0xfc, 0x0b, 0x19, 0x2e, 0x98, 0x08, 0x79, 0x74, 0xf3, - 0x7c, 0xe3, 0x78, 0xcb, 0xe4, 0x36, 0x22, 0xe9, 0x52, 0xb4, 0xbe, 0x5f, 0x6a, 0xf3, 0x7e, 0x25, - 0xd9, 0x84, 0xc3, 0x6c, 0xd7, 0x76, 0x27, 0xcb, 0x38, 0xd6, 0x80, 0x8c, 0x31, 0x40, 0x8e, 0xc2, - 0x12, 0xb6, 0x83, 0xcb, 0x60, 0xd4, 0x08, 0x19, 0xd9, 0x0e, 0x4a, 0x4b, 0x0c, 0x02, 0x2f, 0x88, - 0x92, 0x51, 0xe9, 0x62, 0x20, 0xef, 0x20, 0xeb, 0xa0, 0x60, 0x8f, 0x4c, 0x30, 0x3d, 0x13, 0x75, - 0x5b, 0x7d, 0xad, 0xdb, 0xda, 0xcd, 0x52, 0x6c, 0xba, 0x22, 0x78, 0xa1, 0xf1, 0x6e, 0xf1, 0x2d, - 0xe4, 0x12, 0x14, 0xd1, 0x20, 0x2d, 0xdb, 0x5c, 0xf4, 0x2c, 0x1f, 0xe5, 0x0b, 0x3c, 0xb3, 0x59, - 0x88, 0xcb, 0x86, 0x17, 0xc3, 0x65, 0xea, 0x8d, 0x62, 0x68, 0x90, 0xbf, 0x46, 0x36, 0x13, 0xd3, - 0xf6, 0x14, 0x1f, 0x9e, 0x28, 0xce, 0x8d, 0x31, 0x14, 0x12, 0x08, 0xf7, 0xc9, 0x49, 0x22, 0x41, - 0x35, 0x8e, 0x4a, 0x87, 0x7d, 0x07, 0x39, 0x67, 0x93, 0x95, 0xf1, 0x6a, 0x94, 0x81, 0xf8, 0x88, - 0x81, 0xf5, 0xe0, 0x85, 0xae, 0x88, 0xf2, 0xda, 0xa3, 0xaa, 0x44, 0xda, 0x12, 0xa8, 0xd6, 0x41, - 0x8d, 0x1b, 0x27, 0x1a, 0x1c, 0x8e, 0x86, 0x3d, 0x73, 0x60, 0xb5, 0xee, 0xda, 0x3d, 0x73, 0xa4, - 0xed, 0x48, 0xa4, 0x6f, 0x36, 0x7b, 0xef, 0x57, 0x88, 0x52, 0x45, 0xc8, 0xae, 0xba, 0x26, 0x87, - 0x90, 0x6d, 0x35, 0x47, 0xed, 0xeb, 0xee, 0xa0, 0xa3, 0xed, 0x90, 0x02, 0x1c, 0x0c, 0x86, 0x56, - 0x0c, 0x28, 0x04, 0x20, 0xd3, 0xe9, 0x0f, 0x5b, 0xcd, 0xbe, 0x96, 0x22, 0x7f, 0xc0, 0xf1, 0xd5, - 0x1d, 0x6d, 0x8e, 0xba, 0xc3, 0x81, 0xd5, 0xbd, 0xb5, 0x3a, 0xd4, 0xec, 0x0c, 0x69, 0xb7, 0x39, - 0xd0, 0x76, 0xc9, 0x6f, 0x50, 0xa0, 0xe6, 0xad, 0x39, 0xb2, 0xa8, 0x79, 0xd3, 0xec, 0x0e, 0xe4, - 0x6e, 0xb6, 0xfa, 0x0f, 0x64, 0x16, 0xbf, 0x06, 0x69, 0x7b, 0x37, 0xb8, 0x32, 0xa9, 0xd5, 0xef, - 0xde, 0x74, 0xe5, 0x3b, 0xe5, 0x01, 0x86, 0xf7, 0xf1, 0xac, 0x34, 0xbe, 0x29, 0x90, 0xba, 0x3f, - 0x27, 0x3e, 0xe4, 0x12, 0x7f, 0x5b, 0xf2, 0xf7, 0x56, 0x87, 0xdb, 0x5f, 0x88, 0x62, 0xf9, 0x75, - 0x01, 0xf7, 0x8d, 0xd2, 0xa7, 0xaf, 0xdf, 0x3f, 0xa7, 0x4e, 0x8c, 0xa3, 0xfa, 0xf3, 0x79, 0x3d, - 0x41, 0x5f, 0x2a, 0x55, 0x82, 0x70, 0xb0, 0xd1, 0x0f, 0x39, 0xdb, 0xb2, 0x4b, 0xb6, 0x59, 0xfc, - 0xeb, 0x35, 0x9a, 0xfb, 0xc6, 0x69, 0x74, 0xd6, 0x11, 0x29, 0xc8, 0xb3, 0x36, 0xc8, 0x56, 0xe1, - 0x03, 0xac, 0xd7, 0x3e, 0x2a, 0xca, 0x38, 0x13, 0x7d, 0xf4, 0x2e, 0x7e, 0x06, 0x00, 0x00, 0xff, - 0xff, 0x99, 0xdd, 0x6d, 0x83, 0x35, 0x05, 0x00, 0x00, -} diff --git a/gubernator.pb.gw.go b/gubernator.pb.gw.go index 114eefda..1969cd57 100644 --- a/gubernator.pb.gw.go +++ b/gubernator.pb.gw.go @@ -9,24 +9,27 @@ It translates gRPC into RESTful JSON APIs. package gubernator import ( + "context" "io" "net/http" + "github.com/golang/protobuf/descriptor" "github.com/golang/protobuf/proto" "github.com/grpc-ecosystem/grpc-gateway/runtime" "github.com/grpc-ecosystem/grpc-gateway/utilities" - "golang.org/x/net/context" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/status" ) +// Suppress "imported and not used" errors var _ codes.Code var _ io.Reader var _ status.Status var _ = runtime.String var _ = utilities.NewDoubleArray +var _ = descriptor.ForMessage func request_V1_GetRateLimits_0(ctx context.Context, marshaler runtime.Marshaler, client V1Client, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq GetRateLimitsReq @@ -45,6 +48,23 @@ func request_V1_GetRateLimits_0(ctx context.Context, marshaler runtime.Marshaler } +func local_request_V1_GetRateLimits_0(ctx context.Context, marshaler runtime.Marshaler, server V1Server, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRateLimitsReq + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetRateLimits(ctx, &protoReq) + return msg, metadata, err + +} + func request_V1_HealthCheck_0(ctx context.Context, marshaler runtime.Marshaler, client V1Client, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq HealthCheckReq var metadata runtime.ServerMetadata @@ -54,6 +74,63 @@ func request_V1_HealthCheck_0(ctx context.Context, marshaler runtime.Marshaler, } +func local_request_V1_HealthCheck_0(ctx context.Context, marshaler runtime.Marshaler, server V1Server, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq HealthCheckReq + var metadata runtime.ServerMetadata + + msg, err := server.HealthCheck(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterV1HandlerServer registers the http handlers for service V1 to "mux". +// UnaryRPC :call V1Server directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +func RegisterV1HandlerServer(ctx context.Context, mux *runtime.ServeMux, server V1Server) error { + + mux.Handle("POST", pattern_V1_GetRateLimits_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_V1_GetRateLimits_0(rctx, inboundMarshaler, server, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_V1_GetRateLimits_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_V1_HealthCheck_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_V1_HealthCheck_0(rctx, inboundMarshaler, server, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_V1_HealthCheck_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + // RegisterV1HandlerFromEndpoint is same as RegisterV1Handler but // automatically dials to "endpoint" and closes the connection when "ctx" gets done. func RegisterV1HandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { @@ -136,9 +213,9 @@ func RegisterV1HandlerClient(ctx context.Context, mux *runtime.ServeMux, client } var ( - pattern_V1_GetRateLimits_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"v1", "GetRateLimits"}, "")) + pattern_V1_GetRateLimits_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"v1", "GetRateLimits"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_V1_HealthCheck_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"v1", "HealthCheck"}, "")) + pattern_V1_HealthCheck_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"v1", "HealthCheck"}, "", runtime.AssumeColonVerbOpt(true))) ) var ( diff --git a/hash.go b/hash.go index be632ffa..74a9a099 100644 --- a/hash.go +++ b/hash.go @@ -63,7 +63,7 @@ func (ch *ConsistantHash) Peers() []*PeerClient { // Adds a peer to the hash func (ch *ConsistantHash) Add(peer *PeerClient) { - hash := int(ch.hashFunc(bytes(peer.host))) + hash := int(ch.hashFunc(strToBytesUnsafe(peer.info.HashKey()))) ch.peerKeys = append(ch.peerKeys, hash) ch.peerMap[hash] = peer sort.Ints(ch.peerKeys) @@ -74,9 +74,9 @@ func (ch *ConsistantHash) Size() int { return len(ch.peerKeys) } -// Returns the peer by hostname -func (ch *ConsistantHash) GetPeerByHost(host string) *PeerClient { - return ch.peerMap[int(ch.hashFunc(bytes(host)))] +// Returns the peer by peer info +func (ch *ConsistantHash) GetByPeerInfo(peer PeerInfo) *PeerClient { + return ch.peerMap[int(ch.hashFunc(strToBytesUnsafe(peer.HashKey())))] } // Given a key, return the peer that key is assigned too @@ -85,7 +85,7 @@ func (ch *ConsistantHash) Get(key string) (*PeerClient, error) { return nil, errors.New("unable to pick a peer; pool is empty") } - hash := int(ch.hashFunc(bytes(key))) + hash := int(ch.hashFunc(strToBytesUnsafe(key))) // Binary search for appropriate peer idx := sort.Search(len(ch.peerKeys), func(i int) bool { return ch.peerKeys[i] >= hash }) @@ -100,7 +100,7 @@ func (ch *ConsistantHash) Get(key string) (*PeerClient, error) { // unsafely return the underlying bytes of a string // the caller cannot alter the returned byte slice -func bytes(str string) []byte { +func strToBytesUnsafe(str string) []byte { hdr := *(*reflect.StringHeader)(unsafe.Pointer(&str)) return *(*[]byte)(unsafe.Pointer(&reflect.SliceHeader{ Data: hdr.Data, diff --git a/hash_test.go b/hash_test.go index 676b098f..02d26c66 100644 --- a/hash_test.go +++ b/hash_test.go @@ -23,14 +23,14 @@ func TestConsistantHash(t *testing.T) { } hash := NewConsistantHash(nil) for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) } for input, addr := range cases { t.Run(input, func(t *testing.T) { peer, err := hash.Get(input) assert.Nil(t, err) - assert.Equal(t, addr, peer.host) + assert.Equal(t, addr, peer.info.Address) }) } @@ -40,7 +40,7 @@ func TestConsistantHash(t *testing.T) { hash := NewConsistantHash(nil) for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) } assert.Equal(t, len(hosts), hash.Size()) @@ -51,13 +51,13 @@ func TestConsistantHash(t *testing.T) { hostMap := map[string]*PeerClient{} for _, h := range hosts { - peer := &PeerClient{host: h} + peer := &PeerClient{info: PeerInfo{Address: h}} hash.Add(peer) hostMap[h] = peer } for host, peer := range hostMap { - assert.Equal(t, peer, hash.GetPeerByHost(host)) + assert.Equal(t, peer, hash.GetByPeerInfo(PeerInfo{Address: host})) } }) @@ -85,13 +85,13 @@ func TestConsistantHash(t *testing.T) { hostMap := map[string]int{} for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) hostMap[h] = 0 } for i := range strings { peer, _ := hash.Get(strings[i]) - hostMap[peer.host]++ + hostMap[peer.info.Address]++ } for host, a := range hostMap { @@ -120,7 +120,7 @@ func BenchmarkConsistantHash(b *testing.B) { hash := NewConsistantHash(hashFunc) hosts := []string{"a.svc.local", "b.svc.local", "c.svc.local"} for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) } b.ResetTimer() diff --git a/interval.go b/interval.go index c6985ad3..39e9d1da 100644 --- a/interval.go +++ b/interval.go @@ -36,7 +36,7 @@ type Interval struct { func NewInterval(d time.Duration) *Interval { i := Interval{ C: make(chan struct{}, 1), - in: make(chan struct{}), + in: make(chan struct{}, 1), } i.run(d) return &i diff --git a/memberlist.go b/memberlist.go new file mode 100644 index 00000000..171c1fee --- /dev/null +++ b/memberlist.go @@ -0,0 +1,226 @@ +package gubernator + +import ( + "bytes" + "encoding/gob" + "fmt" + "io" + l "log" + "strconv" + "strings" + "time" + + ml "github.com/hashicorp/memberlist" + "github.com/pkg/errors" +) + +type MemberlistPool struct { + memberlist *ml.Memberlist + conf MemberlistPoolConfig + events *memberlistEventHandler +} + +type MemberlistPoolConfig struct { + AdvertiseAddress string + AdvertisePort int + NodeName string + KnownNodes []string + LoggerOutput io.Writer + Logger *l.Logger + DataCenter string + GubernatorPort int + OnUpdate UpdateFunc + Enabled bool +} + +func NewMemberlistPool(conf MemberlistPoolConfig) (*MemberlistPool, error) { + memberlistPool := &MemberlistPool{conf: conf} + + // Configure memberlist event handler + memberlistPool.events = newMemberListEventHandler(conf.OnUpdate) + + // Configure memberlist + config := ml.DefaultWANConfig() + config.Events = memberlistPool.events + config.AdvertiseAddr = conf.AdvertiseAddress + config.AdvertisePort = conf.AdvertisePort + + if conf.NodeName != "" { + config.Name = conf.NodeName + } + + if conf.LoggerOutput != nil { + config.LogOutput = conf.LoggerOutput + } + + if conf.Logger != nil { + config.Logger = conf.Logger + } + + // Create and set memberlist + memberlist, err := ml.Create(config) + if err != nil { + return nil, err + } + memberlistPool.memberlist = memberlist + + // Prep metadata + gob.Register(memberlistMetadata{}) + metadata := memberlistMetadata{DataCenter: conf.DataCenter, GubernatorPort: conf.GubernatorPort} + + // Join memberlist pool + err = memberlistPool.joinPool(conf.KnownNodes, metadata) + if err != nil { + return nil, err + } + + return memberlistPool, nil +} + +func (m *MemberlistPool) joinPool(knownNodes []string, metadata memberlistMetadata) error { + // Get local node and set metadata + node := m.memberlist.LocalNode() + serializedMetadata, err := serializeMemberlistMetadata(metadata) + if err != nil { + return err + } + node.Meta = serializedMetadata + + // Join memberlist + _, err = m.memberlist.Join(knownNodes) + if err != nil { + return errors.Wrap(err, "while joining memberlist") + } + + // Add the local node to the event handler's peer list + m.events.addPeer(node) + + return nil +} + +func (m *MemberlistPool) Close() { + err := m.memberlist.Leave(time.Second) + if err != nil { + log.Warn(errors.Wrap(err, "while leaving memberlist")) + } +} + +type memberlistEventHandler struct { + peers map[string]PeerInfo + OnUpdate UpdateFunc +} + +func newMemberListEventHandler(onUpdate UpdateFunc) *memberlistEventHandler { + eventhandler := memberlistEventHandler{OnUpdate: onUpdate} + eventhandler.peers = make(map[string]PeerInfo) + return &eventhandler +} + +func (e *memberlistEventHandler) addPeer(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + metadata, err := deserializeMemberlistMetadata(node.Meta) + if err != nil { + log.Warn(errors.Wrap(err, "while adding to peers")) + } else { + // Construct Gubernator address and create PeerInfo + gubernatorAddress := makeAddress(ip, metadata.GubernatorPort) + e.peers[ip] = PeerInfo{Address: gubernatorAddress, DataCenter: metadata.DataCenter} + e.callOnUpdate() + } +} + +func (e *memberlistEventHandler) NotifyJoin(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + metadata, err := deserializeMemberlistMetadata(node.Meta) + if err != nil { + // This is called during memberlist initialization due to the fact that the local node + // has no metadata yet + log.Warn(errors.Wrap(err, "while joining memberlist")) + } else { + // Construct Gubernator address and create PeerInfo + gubernatorAddress := makeAddress(ip, metadata.GubernatorPort) + e.peers[ip] = PeerInfo{Address: gubernatorAddress, DataCenter: metadata.DataCenter} + e.callOnUpdate() + } +} + +func (e *memberlistEventHandler) NotifyLeave(node *ml.Node) { + ip := getIP(node.Address()) + + // Remove PeerInfo + delete(e.peers, ip) + + e.callOnUpdate() +} + +func (e *memberlistEventHandler) NotifyUpdate(node *ml.Node) { + ip := getIP(node.Address()) + + // Deserialize metadata + metadata, err := deserializeMemberlistMetadata(node.Meta) + if err != nil { + log.Warn(errors.Wrap(err, "while updating memberlist")) + } else { + // Construct Gubernator address and create PeerInfo + gubernatorAddress := makeAddress(ip, metadata.GubernatorPort) + e.peers[ip] = PeerInfo{Address: gubernatorAddress, DataCenter: metadata.DataCenter} + e.callOnUpdate() + } +} + +func (e *memberlistEventHandler) callOnUpdate() { + var peers = []PeerInfo{} + + for _, p := range e.peers { + peers = append(peers, p) + } + + e.OnUpdate(peers) +} + +func getIP(address string) string { + return strings.Split(address, ":")[0] +} + +func makeAddress(ip string, port int) string { + return fmt.Sprintf("%s:%s", ip, strconv.Itoa(port)) +} + +type memberlistMetadata struct { + DataCenter string + GubernatorPort int +} + +func serializeMemberlistMetadata(metadata memberlistMetadata) ([]byte, error) { + buf := bytes.Buffer{} + encoder := gob.NewEncoder(&buf) + + err := encoder.Encode(metadata) + if err != nil { + log.Warn(errors.Wrap(err, "error encoding")) + return nil, err + } + + return buf.Bytes(), nil +} + +func deserializeMemberlistMetadata(metadataAsByteSlice []byte) (*memberlistMetadata, error) { + metadata := memberlistMetadata{} + buf := bytes.Buffer{} + + buf.Write(metadataAsByteSlice) + + decoder := gob.NewDecoder(&buf) + + err := decoder.Decode(&metadata) + if err != nil { + log.Warn(errors.Wrap(err, "error decoding")) + return nil, err + } + + return &metadata, nil +} diff --git a/multiregion.go b/multiregion.go new file mode 100644 index 00000000..5843ca7c --- /dev/null +++ b/multiregion.go @@ -0,0 +1,82 @@ +package gubernator + +import ( + "github.com/mailgun/holster/v3/syncutil" + "github.com/sirupsen/logrus" +) + +type mutliRegionManager struct { + reqQueue chan *RateLimitReq + wg syncutil.WaitGroup + conf BehaviorConfig + log *logrus.Entry + instance *Instance +} + +func newMultiRegionManager(conf BehaviorConfig, instance *Instance) *mutliRegionManager { + mm := mutliRegionManager{ + conf: conf, + instance: instance, + reqQueue: make(chan *RateLimitReq, 0), + } + mm.runAsyncReqs() + return &mm +} + +// QueueHits writes the RateLimitReq to be asynchronously sent to other regions +func (mm *mutliRegionManager) QueueHits(r *RateLimitReq) { + mm.reqQueue <- r +} + +func (mm *mutliRegionManager) runAsyncReqs() { + var interval = NewInterval(mm.conf.MultiRegionSyncWait) + hits := make(map[string]*RateLimitReq) + + mm.wg.Until(func(done chan struct{}) bool { + select { + case r := <-mm.reqQueue: + key := r.HashKey() + + // Aggregate the hits into a single request + _, ok := hits[key] + if ok { + hits[key].Hits += r.Hits + } else { + hits[key] = r + } + + // Send the hits if we reached our batch limit + if len(hits) == mm.conf.MultiRegionBatchLimit { + for dc, picker := range mm.instance.GetRegionPickers() { + log.Infof("Sending %v hit(s) to %s picker", len(hits), dc) + mm.sendHits(hits, picker) + } + hits = make(map[string]*RateLimitReq) + } + + // Queue next interval + if len(hits) == 1 { + interval.Next() + } + + case <-interval.C: + if len(hits) > 0 { + for dc, picker := range mm.instance.GetRegionPickers() { + log.Infof("Sending %v hit(s) to %s picker", len(hits), dc) + mm.sendHits(hits, picker) + } + hits = make(map[string]*RateLimitReq) + } + + case <-done: + return false + } + return true + }) +} + +// TODO: Sending cross DC should mainly update the hits, the config should not be sent, or ignored when received +// TODO: Calculation of OVERLIMIT should not occur when sending hits cross DC +func (mm *mutliRegionManager) sendHits(r map[string]*RateLimitReq, picker PeerPicker) { + // Does nothing for now +} diff --git a/peers.go b/peer_client.go similarity index 58% rename from peers.go rename to peer_client.go index 4cd1ea98..49e69a8d 100644 --- a/peers.go +++ b/peer_client.go @@ -18,35 +18,43 @@ package gubernator import ( "context" + "fmt" "sync" + "time" + "github.com/mailgun/holster/v3/collections" "github.com/pkg/errors" "google.golang.org/grpc" ) -// ErrClosing is the error returned when the client is closing -var ErrClosing = errors.New("closing") - type PeerPicker interface { - GetPeerByHost(host string) *PeerClient + GetByPeerInfo(PeerInfo) *PeerClient Peers() []*PeerClient Get(string) (*PeerClient, error) New() PeerPicker Add(*PeerClient) - Size() int + Size() int // TODO: Might not be useful? } +type peerStatus int + +const ( + peerNotConnected peerStatus = iota + peerConnected + peerClosing +) + type PeerClient struct { - client PeersV1Client - conn *grpc.ClientConn - conf BehaviorConfig - queue chan *request - host string - isOwner bool // true if this peer refers to this server instance - - mutex sync.RWMutex // This mutex is for verifying the closing state of the client - isClosing bool - wg sync.WaitGroup // This wait group is to monitor the number of in-flight requests + client PeersV1Client + conn *grpc.ClientConn + conf BehaviorConfig + queue chan *request + info PeerInfo + lastErrs *collections.LRUCache + + mutex sync.RWMutex // This mutex is for verifying the closing state of the client + status peerStatus // Keep the current status of the peer + wg sync.WaitGroup // This wait group is to monitor the number of in-flight requests } type response struct { @@ -59,20 +67,59 @@ type request struct { resp chan *response } -func NewPeerClient(conf BehaviorConfig, host string) (*PeerClient, error) { - c := &PeerClient{ - queue: make(chan *request, 1000), - host: host, - conf: conf, +func NewPeerClient(conf BehaviorConfig, info PeerInfo) *PeerClient { + return &PeerClient{ + queue: make(chan *request, 1000), + status: peerNotConnected, + conf: conf, + info: info, + lastErrs: collections.NewLRUCache(100), } +} - if err := c.dialPeer(); err != nil { - return nil, err +// Connect establishes a GRPC connection to a peer +func (c *PeerClient) connect() error { + // NOTE: To future self, this mutex is used here because we need to know if the peer is disconnecting and + // handle ErrClosing. Since this mutex MUST be here we take this opportunity to also see if we are connected. + // Doing this here encapsulates managing the connected state to the PeerClient struct. Previously a PeerClient + // was connected when `NewPeerClient()` was called however, when adding support for multi data centers having a + // PeerClient connected to every Peer in every data center continuously is not desirable, especially if nodes + // in each region are configured to all have sisters. + + c.mutex.RLock() + if c.status == peerClosing { + c.mutex.RUnlock() + return &PeerErr{err: errors.New("already disconnecting")} } - go c.run() + if c.status == peerNotConnected { + // This mutex stuff looks wonky, but it allows us to use RLock() 99% of the time, while the 1% where we + // actually need to connect uses a full Lock(), using RLock() most of which should reduce the over head + // of a full lock on every call - return c, nil + // Yield the read lock so we can get the RW lock + c.mutex.RUnlock() + c.mutex.Lock() + defer c.mutex.Unlock() + + // Now that we have the RW lock, ensure no else got here ahead of us. + if c.status == peerConnected { + return nil + } + + var err error + // c.conn, err = grpc.Dial(fmt.Sprintf("%s:%s", c.info.Address, ""), grpc.WithInsecure()) + c.conn, err = grpc.Dial(c.info.Address, grpc.WithInsecure()) + if err != nil { + return c.setLastErr(&PeerErr{err: errors.Wrapf(err, "failed to dial peer %s", c.info.Address)}) + } + c.client = NewPeersV1Client(c.conn) + c.status = peerConnected + go c.run() + return nil + } + c.mutex.RUnlock() + return nil } // GetPeerRateLimit forwards a rate limit request to a peer. If the rate limit has `behavior == BATCHING` configured @@ -85,7 +132,7 @@ func (c *PeerClient) GetPeerRateLimit(ctx context.Context, r *RateLimitReq) (*Ra Requests: []*RateLimitReq{r}, }) if err != nil { - return nil, err + return nil, c.setLastErr(err) } return resp.RateLimits[0], nil } @@ -94,10 +141,8 @@ func (c *PeerClient) GetPeerRateLimit(ctx context.Context, r *RateLimitReq) (*Ra // GetPeerRateLimits requests a list of rate limit statuses from a peer func (c *PeerClient) GetPeerRateLimits(ctx context.Context, r *GetPeerRateLimitsReq) (*GetPeerRateLimitsResp, error) { - c.mutex.RLock() - if c.isClosing { - c.mutex.RUnlock() - return nil, ErrClosing + if err := c.connect(); err != nil { + return nil, err } // NOTE: This must be done within the RLock since calling Wait() in Shutdown() causes @@ -106,11 +151,9 @@ func (c *PeerClient) GetPeerRateLimits(ctx context.Context, r *GetPeerRateLimits c.wg.Add(1) defer c.wg.Done() - c.mutex.RUnlock() - resp, err := c.client.GetPeerRateLimits(ctx, r) if err != nil { - return nil, err + return nil, c.setLastErr(err) } // Unlikely, but this avoids a panic if something wonky happens @@ -122,26 +165,56 @@ func (c *PeerClient) GetPeerRateLimits(ctx context.Context, r *GetPeerRateLimits // UpdatePeerGlobals sends global rate limit status updates to a peer func (c *PeerClient) UpdatePeerGlobals(ctx context.Context, r *UpdatePeerGlobalsReq) (*UpdatePeerGlobalsResp, error) { - c.mutex.RLock() - if c.isClosing { - c.mutex.RUnlock() - return nil, ErrClosing + if err := c.connect(); err != nil { + return nil, err } // See NOTE above about RLock and wg.Add(1) c.wg.Add(1) defer c.wg.Done() - c.mutex.RUnlock() + resp, err := c.client.UpdatePeerGlobals(ctx, r) + if err != nil { + c.setLastErr(err) + } - return c.client.UpdatePeerGlobals(ctx, r) + return resp, err +} + +func (c *PeerClient) setLastErr(err error) error { + // If we get a nil error return without caching it + if err == nil { + return err + } + + // Prepend client address to error + errWithHostname := errors.Wrap(err, fmt.Sprintf("from host %s", c.info.Address)) + key := err.Error() + + // Add error to the cache with a TTL of 5 minutes + c.lastErrs.AddWithTTL(key, errWithHostname, time.Minute*5) + + return err +} + +func (c *PeerClient) GetLastErr() []string { + var errs []string + keys := c.lastErrs.Keys() + + // Get errors from each key in the cache + for _, key := range keys { + err, ok := c.lastErrs.Get(key) + if ok { + errs = append(errs, err.(error).Error()) + } + } + + return errs } func (c *PeerClient) getPeerRateLimitsBatch(ctx context.Context, r *RateLimitReq) (*RateLimitResp, error) { - c.mutex.RLock() - if c.isClosing { - c.mutex.RUnlock() - return nil, ErrClosing + if err := c.connect(); err != nil { + return nil, err } req := request{request: r, resp: make(chan *response, 1)} @@ -153,33 +226,18 @@ func (c *PeerClient) getPeerRateLimitsBatch(ctx context.Context, r *RateLimitReq c.wg.Add(1) defer c.wg.Done() - // Unlock to prevent the chan from being closed - c.mutex.RUnlock() - // Wait for a response or context cancel select { case resp := <-req.resp: if resp.err != nil { - return nil, resp.err + return nil, c.setLastErr(resp.err) } return resp.rl, nil case <-ctx.Done(): - return nil, ctx.Err() + return nil, c.setLastErr(ctx.Err()) } } -// dialPeer dials a peer and initializes the GRPC client -func (c *PeerClient) dialPeer() error { - var err error - c.conn, err = grpc.Dial(c.host, grpc.WithInsecure()) - if err != nil { - return errors.Wrapf(err, "failed to dial peer %s", c.host) - } - - c.client = NewPeersV1Client(c.conn) - return nil -} - // run waits for requests to be queued, when either c.batchWait time // has elapsed or the queue reaches c.batchLimit. Send what is in the queue. func (c *PeerClient) run() { @@ -238,6 +296,7 @@ func (c *PeerClient) sendQueue(queue []*request) { // An error here indicates the entire request failed if err != nil { + c.setLastErr(err) for _, r := range queue { r.resp <- &response{err: err} } @@ -263,13 +322,13 @@ func (c *PeerClient) sendQueue(queue []*request) { func (c *PeerClient) Shutdown(ctx context.Context) error { // Take the write lock since we're going to modify the closing state c.mutex.Lock() - if c.isClosing { + if c.status == peerClosing || c.status == peerNotConnected { c.mutex.Unlock() return nil } defer c.mutex.Unlock() - c.isClosing = true + c.status = peerClosing // We need to close the chan here to prevent a possible race close(c.queue) @@ -295,3 +354,30 @@ func (c *PeerClient) Shutdown(ctx context.Context) error { return nil } } + +// PeerErr is returned if the peer is not connected or is in a closing state +type PeerErr struct { + err error +} + +func (p *PeerErr) NotReady() bool { + return true +} + +func (p *PeerErr) Error() string { + return p.err.Error() +} + +func (p *PeerErr) Cause() error { + return p.err +} + +type notReadyErr interface { + NotReady() bool +} + +// IsNotReady returns true if the err is because the peer is not connected or in a closing state +func IsNotReady(err error) bool { + te, ok := err.(notReadyErr) + return ok && te.NotReady() +} diff --git a/peers_test.go b/peer_client_test.go similarity index 81% rename from peers_test.go rename to peer_client_test.go index 50d25bfc..af0e3f1e 100644 --- a/peers_test.go +++ b/peer_client_test.go @@ -41,8 +41,7 @@ func TestPeerClientShutdown(t *testing.T) { t.Run(c.Name, func(t *testing.T) { - client, err := gubernator.NewPeerClient(config, cluster.GetPeer()) - assert.NoError(t, err) + client := gubernator.NewPeerClient(config, cluster.GetRandomPeer()) wg := sync.WaitGroup{} wg.Add(threads) @@ -57,14 +56,24 @@ func TestPeerClientShutdown(t *testing.T) { Behavior: c.Behavior, }) - assert.Contains(t, []error{nil, gubernator.ErrClosing}, err) + isExpectedErr := false + + switch err.(type) { + case *gubernator.PeerErr: + isExpectedErr = true + case nil: + isExpectedErr = true + } + + assert.True(t, true, isExpectedErr) + }() } // yield the processor that way we allow other goroutines to start their request runtime.Gosched() - err = client.Shutdown(context.Background()) + err := client.Shutdown(context.Background()) assert.NoError(t, err) wg.Wait() diff --git a/peers.pb.go b/peers.pb.go index 37db1803..464e8609 100644 --- a/peers.pb.go +++ b/peers.pb.go @@ -3,13 +3,14 @@ package gubernator -import proto "github.com/golang/protobuf/proto" -import fmt "fmt" -import math "math" - import ( - context "golang.org/x/net/context" + context "context" + fmt "fmt" + proto "github.com/golang/protobuf/proto" grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + math "math" ) // Reference imports to suppress errors if they are not otherwise used. @@ -17,16 +18,45 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package + type GetPeerRateLimitsReq struct { // Must specify at least one RateLimit. The peer that recives this request MUST be authoritative for // each rate_limit[x].unique_key provided, as the peer will not forward the request to any other peers - Requests []*RateLimitReq `protobuf:"bytes,1,rep,name=requests" json:"requests,omitempty"` + Requests []*RateLimitReq `protobuf:"bytes,1,rep,name=requests,proto3" json:"requests,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GetPeerRateLimitsReq) Reset() { *m = GetPeerRateLimitsReq{} } -func (m *GetPeerRateLimitsReq) String() string { return proto.CompactTextString(m) } -func (*GetPeerRateLimitsReq) ProtoMessage() {} -func (*GetPeerRateLimitsReq) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{0} } +func (m *GetPeerRateLimitsReq) Reset() { *m = GetPeerRateLimitsReq{} } +func (m *GetPeerRateLimitsReq) String() string { return proto.CompactTextString(m) } +func (*GetPeerRateLimitsReq) ProtoMessage() {} +func (*GetPeerRateLimitsReq) Descriptor() ([]byte, []int) { + return fileDescriptor_9b029f9e0d26cab5, []int{0} +} + +func (m *GetPeerRateLimitsReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetPeerRateLimitsReq.Unmarshal(m, b) +} +func (m *GetPeerRateLimitsReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetPeerRateLimitsReq.Marshal(b, m, deterministic) +} +func (m *GetPeerRateLimitsReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetPeerRateLimitsReq.Merge(m, src) +} +func (m *GetPeerRateLimitsReq) XXX_Size() int { + return xxx_messageInfo_GetPeerRateLimitsReq.Size(m) +} +func (m *GetPeerRateLimitsReq) XXX_DiscardUnknown() { + xxx_messageInfo_GetPeerRateLimitsReq.DiscardUnknown(m) +} + +var xxx_messageInfo_GetPeerRateLimitsReq proto.InternalMessageInfo func (m *GetPeerRateLimitsReq) GetRequests() []*RateLimitReq { if m != nil { @@ -37,13 +67,36 @@ func (m *GetPeerRateLimitsReq) GetRequests() []*RateLimitReq { type GetPeerRateLimitsResp struct { // Responses are in the same order as they appeared in the PeerRateLimitRequests - RateLimits []*RateLimitResp `protobuf:"bytes,1,rep,name=rate_limits,json=rateLimits" json:"rate_limits,omitempty"` + RateLimits []*RateLimitResp `protobuf:"bytes,1,rep,name=rate_limits,json=rateLimits,proto3" json:"rate_limits,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GetPeerRateLimitsResp) Reset() { *m = GetPeerRateLimitsResp{} } -func (m *GetPeerRateLimitsResp) String() string { return proto.CompactTextString(m) } -func (*GetPeerRateLimitsResp) ProtoMessage() {} -func (*GetPeerRateLimitsResp) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{1} } +func (m *GetPeerRateLimitsResp) Reset() { *m = GetPeerRateLimitsResp{} } +func (m *GetPeerRateLimitsResp) String() string { return proto.CompactTextString(m) } +func (*GetPeerRateLimitsResp) ProtoMessage() {} +func (*GetPeerRateLimitsResp) Descriptor() ([]byte, []int) { + return fileDescriptor_9b029f9e0d26cab5, []int{1} +} + +func (m *GetPeerRateLimitsResp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetPeerRateLimitsResp.Unmarshal(m, b) +} +func (m *GetPeerRateLimitsResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetPeerRateLimitsResp.Marshal(b, m, deterministic) +} +func (m *GetPeerRateLimitsResp) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetPeerRateLimitsResp.Merge(m, src) +} +func (m *GetPeerRateLimitsResp) XXX_Size() int { + return xxx_messageInfo_GetPeerRateLimitsResp.Size(m) +} +func (m *GetPeerRateLimitsResp) XXX_DiscardUnknown() { + xxx_messageInfo_GetPeerRateLimitsResp.DiscardUnknown(m) +} + +var xxx_messageInfo_GetPeerRateLimitsResp proto.InternalMessageInfo func (m *GetPeerRateLimitsResp) GetRateLimits() []*RateLimitResp { if m != nil { @@ -54,13 +107,36 @@ func (m *GetPeerRateLimitsResp) GetRateLimits() []*RateLimitResp { type UpdatePeerGlobalsReq struct { // Must specify at least one RateLimit - Globals []*UpdatePeerGlobal `protobuf:"bytes,1,rep,name=globals" json:"globals,omitempty"` + Globals []*UpdatePeerGlobal `protobuf:"bytes,1,rep,name=globals,proto3" json:"globals,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *UpdatePeerGlobalsReq) Reset() { *m = UpdatePeerGlobalsReq{} } -func (m *UpdatePeerGlobalsReq) String() string { return proto.CompactTextString(m) } -func (*UpdatePeerGlobalsReq) ProtoMessage() {} -func (*UpdatePeerGlobalsReq) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{2} } +func (m *UpdatePeerGlobalsReq) Reset() { *m = UpdatePeerGlobalsReq{} } +func (m *UpdatePeerGlobalsReq) String() string { return proto.CompactTextString(m) } +func (*UpdatePeerGlobalsReq) ProtoMessage() {} +func (*UpdatePeerGlobalsReq) Descriptor() ([]byte, []int) { + return fileDescriptor_9b029f9e0d26cab5, []int{2} +} + +func (m *UpdatePeerGlobalsReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_UpdatePeerGlobalsReq.Unmarshal(m, b) +} +func (m *UpdatePeerGlobalsReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_UpdatePeerGlobalsReq.Marshal(b, m, deterministic) +} +func (m *UpdatePeerGlobalsReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdatePeerGlobalsReq.Merge(m, src) +} +func (m *UpdatePeerGlobalsReq) XXX_Size() int { + return xxx_messageInfo_UpdatePeerGlobalsReq.Size(m) +} +func (m *UpdatePeerGlobalsReq) XXX_DiscardUnknown() { + xxx_messageInfo_UpdatePeerGlobalsReq.DiscardUnknown(m) +} + +var xxx_messageInfo_UpdatePeerGlobalsReq proto.InternalMessageInfo func (m *UpdatePeerGlobalsReq) GetGlobals() []*UpdatePeerGlobal { if m != nil { @@ -70,15 +146,38 @@ func (m *UpdatePeerGlobalsReq) GetGlobals() []*UpdatePeerGlobal { } type UpdatePeerGlobal struct { - Key string `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` - Status *RateLimitResp `protobuf:"bytes,2,opt,name=status" json:"status,omitempty"` - Algorithm Algorithm `protobuf:"varint,3,opt,name=algorithm,enum=pb.gubernator.Algorithm" json:"algorithm,omitempty"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Status *RateLimitResp `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + Algorithm Algorithm `protobuf:"varint,3,opt,name=algorithm,proto3,enum=pb.gubernator.Algorithm" json:"algorithm,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UpdatePeerGlobal) Reset() { *m = UpdatePeerGlobal{} } +func (m *UpdatePeerGlobal) String() string { return proto.CompactTextString(m) } +func (*UpdatePeerGlobal) ProtoMessage() {} +func (*UpdatePeerGlobal) Descriptor() ([]byte, []int) { + return fileDescriptor_9b029f9e0d26cab5, []int{3} +} + +func (m *UpdatePeerGlobal) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_UpdatePeerGlobal.Unmarshal(m, b) +} +func (m *UpdatePeerGlobal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_UpdatePeerGlobal.Marshal(b, m, deterministic) +} +func (m *UpdatePeerGlobal) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdatePeerGlobal.Merge(m, src) +} +func (m *UpdatePeerGlobal) XXX_Size() int { + return xxx_messageInfo_UpdatePeerGlobal.Size(m) +} +func (m *UpdatePeerGlobal) XXX_DiscardUnknown() { + xxx_messageInfo_UpdatePeerGlobal.DiscardUnknown(m) } -func (m *UpdatePeerGlobal) Reset() { *m = UpdatePeerGlobal{} } -func (m *UpdatePeerGlobal) String() string { return proto.CompactTextString(m) } -func (*UpdatePeerGlobal) ProtoMessage() {} -func (*UpdatePeerGlobal) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3} } +var xxx_messageInfo_UpdatePeerGlobal proto.InternalMessageInfo func (m *UpdatePeerGlobal) GetKey() string { if m != nil { @@ -102,12 +201,35 @@ func (m *UpdatePeerGlobal) GetAlgorithm() Algorithm { } type UpdatePeerGlobalsResp struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UpdatePeerGlobalsResp) Reset() { *m = UpdatePeerGlobalsResp{} } +func (m *UpdatePeerGlobalsResp) String() string { return proto.CompactTextString(m) } +func (*UpdatePeerGlobalsResp) ProtoMessage() {} +func (*UpdatePeerGlobalsResp) Descriptor() ([]byte, []int) { + return fileDescriptor_9b029f9e0d26cab5, []int{4} } -func (m *UpdatePeerGlobalsResp) Reset() { *m = UpdatePeerGlobalsResp{} } -func (m *UpdatePeerGlobalsResp) String() string { return proto.CompactTextString(m) } -func (*UpdatePeerGlobalsResp) ProtoMessage() {} -func (*UpdatePeerGlobalsResp) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{4} } +func (m *UpdatePeerGlobalsResp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_UpdatePeerGlobalsResp.Unmarshal(m, b) +} +func (m *UpdatePeerGlobalsResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_UpdatePeerGlobalsResp.Marshal(b, m, deterministic) +} +func (m *UpdatePeerGlobalsResp) XXX_Merge(src proto.Message) { + xxx_messageInfo_UpdatePeerGlobalsResp.Merge(m, src) +} +func (m *UpdatePeerGlobalsResp) XXX_Size() int { + return xxx_messageInfo_UpdatePeerGlobalsResp.Size(m) +} +func (m *UpdatePeerGlobalsResp) XXX_DiscardUnknown() { + xxx_messageInfo_UpdatePeerGlobalsResp.DiscardUnknown(m) +} + +var xxx_messageInfo_UpdatePeerGlobalsResp proto.InternalMessageInfo func init() { proto.RegisterType((*GetPeerRateLimitsReq)(nil), "pb.gubernator.GetPeerRateLimitsReq") @@ -117,6 +239,32 @@ func init() { proto.RegisterType((*UpdatePeerGlobalsResp)(nil), "pb.gubernator.UpdatePeerGlobalsResp") } +func init() { proto.RegisterFile("peers.proto", fileDescriptor_9b029f9e0d26cab5) } + +var fileDescriptor_9b029f9e0d26cab5 = []byte{ + // 317 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x92, 0xc1, 0x4a, 0xc3, 0x40, + 0x10, 0x40, 0xbb, 0x16, 0x5a, 0x3b, 0x45, 0xad, 0x4b, 0x8b, 0xa1, 0x0a, 0x86, 0xd5, 0x43, 0x4e, + 0x01, 0xa3, 0x28, 0x1e, 0x3c, 0xe8, 0xa5, 0x17, 0x41, 0x5d, 0xb0, 0x07, 0x2f, 0xba, 0xc1, 0x21, + 0x06, 0x53, 0xb3, 0xd9, 0xdd, 0x1c, 0xbc, 0xf9, 0x0d, 0xfe, 0x97, 0xff, 0x24, 0x49, 0xd3, 0x04, + 0x93, 0x48, 0x6f, 0x9b, 0x9d, 0x37, 0x6f, 0x66, 0x32, 0x0b, 0x43, 0x89, 0xa8, 0xb4, 0x2b, 0x55, + 0x6c, 0x62, 0xba, 0x25, 0x7d, 0x37, 0x48, 0x7d, 0x54, 0x1f, 0xc2, 0xc4, 0x6a, 0x3a, 0xaa, 0xce, + 0x4b, 0x80, 0xdd, 0xc1, 0x78, 0x86, 0xe6, 0x1e, 0x51, 0x71, 0x61, 0xf0, 0x36, 0x5c, 0x84, 0x46, + 0x73, 0x4c, 0xe8, 0x05, 0x6c, 0x2a, 0x4c, 0x52, 0xd4, 0x46, 0x5b, 0xc4, 0xee, 0x3a, 0x43, 0x6f, + 0xdf, 0xfd, 0xe3, 0x72, 0x4b, 0x9e, 0x63, 0xc2, 0x4b, 0x98, 0xcd, 0x61, 0xd2, 0x22, 0xd4, 0x92, + 0x5e, 0xc1, 0x50, 0x09, 0x83, 0xcf, 0x51, 0x7e, 0x55, 0x48, 0x0f, 0xfe, 0x97, 0x6a, 0xc9, 0x41, + 0x95, 0x0a, 0xf6, 0x00, 0xe3, 0x47, 0xf9, 0x2a, 0x0c, 0x66, 0xea, 0x59, 0x14, 0xfb, 0x22, 0xca, + 0x1b, 0xbd, 0x84, 0x7e, 0xb0, 0xfc, 0x2a, 0x94, 0x87, 0x35, 0x65, 0x3d, 0x8b, 0xaf, 0x78, 0xf6, + 0x4d, 0x60, 0x54, 0x8f, 0xd2, 0x11, 0x74, 0xdf, 0xf1, 0xd3, 0x22, 0x36, 0x71, 0x06, 0x3c, 0x3b, + 0xd2, 0x33, 0xe8, 0x69, 0x23, 0x4c, 0xaa, 0xad, 0x0d, 0x9b, 0xac, 0xed, 0xb9, 0x60, 0xe9, 0x39, + 0x0c, 0x44, 0x14, 0xc4, 0x2a, 0x34, 0x6f, 0x0b, 0xab, 0x6b, 0x13, 0x67, 0xdb, 0xb3, 0x6a, 0x89, + 0xd7, 0xab, 0x38, 0xaf, 0x50, 0xb6, 0x07, 0x93, 0x96, 0x39, 0xb5, 0xf4, 0x7e, 0x08, 0xf4, 0xb3, + 0x3b, 0x3d, 0x3f, 0xa1, 0x2f, 0xb0, 0xdb, 0xf8, 0xc9, 0xf4, 0xa8, 0xa6, 0x6f, 0xdb, 0xeb, 0xf4, + 0x78, 0x3d, 0xa4, 0x25, 0xeb, 0x64, 0x15, 0x1a, 0x6d, 0x34, 0x2a, 0xb4, 0x2d, 0xa4, 0x51, 0xa1, + 0x75, 0x1a, 0xd6, 0xb9, 0xd9, 0x79, 0x82, 0x8a, 0xfa, 0x22, 0xc4, 0xef, 0xe5, 0x2f, 0xf2, 0xf4, + 0x37, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x74, 0x68, 0xc5, 0xc1, 0x02, 0x00, 0x00, +} + // Reference imports to suppress errors if they are not otherwise used. var _ context.Context var _ grpc.ClientConn @@ -125,8 +273,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for PeersV1 service - +// PeersV1Client is the client API for PeersV1 service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type PeersV1Client interface { // Used by peers to relay batches of requests to an authoritative peer GetPeerRateLimits(ctx context.Context, in *GetPeerRateLimitsReq, opts ...grpc.CallOption) (*GetPeerRateLimitsResp, error) @@ -144,7 +293,7 @@ func NewPeersV1Client(cc *grpc.ClientConn) PeersV1Client { func (c *peersV1Client) GetPeerRateLimits(ctx context.Context, in *GetPeerRateLimitsReq, opts ...grpc.CallOption) (*GetPeerRateLimitsResp, error) { out := new(GetPeerRateLimitsResp) - err := grpc.Invoke(ctx, "/pb.gubernator.PeersV1/GetPeerRateLimits", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/pb.gubernator.PeersV1/GetPeerRateLimits", in, out, opts...) if err != nil { return nil, err } @@ -153,15 +302,14 @@ func (c *peersV1Client) GetPeerRateLimits(ctx context.Context, in *GetPeerRateLi func (c *peersV1Client) UpdatePeerGlobals(ctx context.Context, in *UpdatePeerGlobalsReq, opts ...grpc.CallOption) (*UpdatePeerGlobalsResp, error) { out := new(UpdatePeerGlobalsResp) - err := grpc.Invoke(ctx, "/pb.gubernator.PeersV1/UpdatePeerGlobals", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/pb.gubernator.PeersV1/UpdatePeerGlobals", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for PeersV1 service - +// PeersV1Server is the server API for PeersV1 service. type PeersV1Server interface { // Used by peers to relay batches of requests to an authoritative peer GetPeerRateLimits(context.Context, *GetPeerRateLimitsReq) (*GetPeerRateLimitsResp, error) @@ -169,6 +317,17 @@ type PeersV1Server interface { UpdatePeerGlobals(context.Context, *UpdatePeerGlobalsReq) (*UpdatePeerGlobalsResp, error) } +// UnimplementedPeersV1Server can be embedded to have forward compatible implementations. +type UnimplementedPeersV1Server struct { +} + +func (*UnimplementedPeersV1Server) GetPeerRateLimits(ctx context.Context, req *GetPeerRateLimitsReq) (*GetPeerRateLimitsResp, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetPeerRateLimits not implemented") +} +func (*UnimplementedPeersV1Server) UpdatePeerGlobals(ctx context.Context, req *UpdatePeerGlobalsReq) (*UpdatePeerGlobalsResp, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdatePeerGlobals not implemented") +} + func RegisterPeersV1Server(s *grpc.Server, srv PeersV1Server) { s.RegisterService(&_PeersV1_serviceDesc, srv) } @@ -225,29 +384,3 @@ var _PeersV1_serviceDesc = grpc.ServiceDesc{ Streams: []grpc.StreamDesc{}, Metadata: "peers.proto", } - -func init() { proto.RegisterFile("peers.proto", fileDescriptor1) } - -var fileDescriptor1 = []byte{ - // 317 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x92, 0xc1, 0x4a, 0xc3, 0x40, - 0x10, 0x40, 0xbb, 0x16, 0x5a, 0x3b, 0x45, 0xad, 0x4b, 0x8b, 0xa1, 0x0a, 0x86, 0xd5, 0x43, 0x4e, - 0x01, 0xa3, 0x28, 0x1e, 0x3c, 0xe8, 0xa5, 0x17, 0x41, 0x5d, 0xb0, 0x07, 0x2f, 0xba, 0xc1, 0x21, - 0x06, 0x53, 0xb3, 0xd9, 0xdd, 0x1c, 0xbc, 0xf9, 0x0d, 0xfe, 0x97, 0xff, 0x24, 0x49, 0xd3, 0x04, - 0x93, 0x48, 0x6f, 0x9b, 0x9d, 0x37, 0x6f, 0x66, 0x32, 0x0b, 0x43, 0x89, 0xa8, 0xb4, 0x2b, 0x55, - 0x6c, 0x62, 0xba, 0x25, 0x7d, 0x37, 0x48, 0x7d, 0x54, 0x1f, 0xc2, 0xc4, 0x6a, 0x3a, 0xaa, 0xce, - 0x4b, 0x80, 0xdd, 0xc1, 0x78, 0x86, 0xe6, 0x1e, 0x51, 0x71, 0x61, 0xf0, 0x36, 0x5c, 0x84, 0x46, - 0x73, 0x4c, 0xe8, 0x05, 0x6c, 0x2a, 0x4c, 0x52, 0xd4, 0x46, 0x5b, 0xc4, 0xee, 0x3a, 0x43, 0x6f, - 0xdf, 0xfd, 0xe3, 0x72, 0x4b, 0x9e, 0x63, 0xc2, 0x4b, 0x98, 0xcd, 0x61, 0xd2, 0x22, 0xd4, 0x92, - 0x5e, 0xc1, 0x50, 0x09, 0x83, 0xcf, 0x51, 0x7e, 0x55, 0x48, 0x0f, 0xfe, 0x97, 0x6a, 0xc9, 0x41, - 0x95, 0x0a, 0xf6, 0x00, 0xe3, 0x47, 0xf9, 0x2a, 0x0c, 0x66, 0xea, 0x59, 0x14, 0xfb, 0x22, 0xca, - 0x1b, 0xbd, 0x84, 0x7e, 0xb0, 0xfc, 0x2a, 0x94, 0x87, 0x35, 0x65, 0x3d, 0x8b, 0xaf, 0x78, 0xf6, - 0x4d, 0x60, 0x54, 0x8f, 0xd2, 0x11, 0x74, 0xdf, 0xf1, 0xd3, 0x22, 0x36, 0x71, 0x06, 0x3c, 0x3b, - 0xd2, 0x33, 0xe8, 0x69, 0x23, 0x4c, 0xaa, 0xad, 0x0d, 0x9b, 0xac, 0xed, 0xb9, 0x60, 0xe9, 0x39, - 0x0c, 0x44, 0x14, 0xc4, 0x2a, 0x34, 0x6f, 0x0b, 0xab, 0x6b, 0x13, 0x67, 0xdb, 0xb3, 0x6a, 0x89, - 0xd7, 0xab, 0x38, 0xaf, 0x50, 0xb6, 0x07, 0x93, 0x96, 0x39, 0xb5, 0xf4, 0x7e, 0x08, 0xf4, 0xb3, - 0x3b, 0x3d, 0x3f, 0xa1, 0x2f, 0xb0, 0xdb, 0xf8, 0xc9, 0xf4, 0xa8, 0xa6, 0x6f, 0xdb, 0xeb, 0xf4, - 0x78, 0x3d, 0xa4, 0x25, 0xeb, 0x64, 0x15, 0x1a, 0x6d, 0x34, 0x2a, 0xb4, 0x2d, 0xa4, 0x51, 0xa1, - 0x75, 0x1a, 0xd6, 0xb9, 0xd9, 0x79, 0x82, 0x8a, 0xfa, 0x22, 0xc4, 0xef, 0xe5, 0x2f, 0xf2, 0xf4, - 0x37, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x74, 0x68, 0xc5, 0xc1, 0x02, 0x00, 0x00, -} diff --git a/proto/gubernator.proto b/proto/gubernator.proto index f0c3f49f..22dcc721 100644 --- a/proto/gubernator.proto +++ b/proto/gubernator.proto @@ -122,6 +122,11 @@ enum Behavior { // cache value. For `LEAKY_BUCKET` it sets the `Remaining` to `Limit`. RESET_REMAINING = 8; + // Enables rate limits to be pushed to other regions. Currently this is only implemented on memberlist + // pools. Also requires GUBER_DATA_CENTER to be set to different values on at least 2 instances of + // Gubernator. + MULTI_REGION = 16; + // TODO: Add support for LOCAL. Which would force the rate limit to be handled by the local instance } diff --git a/python/gubernator/gubernator_pb2.py b/python/gubernator/gubernator_pb2.py index b562c7fb..1f98befd 100644 --- a/python/gubernator/gubernator_pb2.py +++ b/python/gubernator/gubernator_pb2.py @@ -1,3 +1,4 @@ +# -*- coding: utf-8 -*- # Generated by the protocol buffer compiler. DO NOT EDIT! # source: gubernator.proto @@ -21,7 +22,7 @@ package='pb.gubernator', syntax='proto3', serialized_options=_b('Z\ngubernator\200\001\001'), - serialized_pb=_b('\n\x10gubernator.proto\x12\rpb.gubernator\x1a\x1cgoogle/api/annotations.proto\"A\n\x10GetRateLimitsReq\x12-\n\x08requests\x18\x01 \x03(\x0b\x32\x1b.pb.gubernator.RateLimitReq\"D\n\x11GetRateLimitsResp\x12/\n\tresponses\x18\x01 \x03(\x0b\x32\x1c.pb.gubernator.RateLimitResp\"\xb7\x01\n\x0cRateLimitReq\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x12\n\nunique_key\x18\x02 \x01(\t\x12\x0c\n\x04hits\x18\x03 \x01(\x03\x12\r\n\x05limit\x18\x04 \x01(\x03\x12\x10\n\x08\x64uration\x18\x05 \x01(\x03\x12+\n\talgorithm\x18\x06 \x01(\x0e\x32\x18.pb.gubernator.Algorithm\x12)\n\x08\x62\x65havior\x18\x07 \x01(\x0e\x32\x17.pb.gubernator.Behavior\"\xea\x01\n\rRateLimitResp\x12%\n\x06status\x18\x01 \x01(\x0e\x32\x15.pb.gubernator.Status\x12\r\n\x05limit\x18\x02 \x01(\x03\x12\x11\n\tremaining\x18\x03 \x01(\x03\x12\x12\n\nreset_time\x18\x04 \x01(\x03\x12\r\n\x05\x65rror\x18\x05 \x01(\t\x12<\n\x08metadata\x18\x06 \x03(\x0b\x32*.pb.gubernator.RateLimitResp.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x10\n\x0eHealthCheckReq\"F\n\x0fHealthCheckResp\x12\x0e\n\x06status\x18\x01 \x01(\t\x12\x0f\n\x07message\x18\x02 \x01(\t\x12\x12\n\npeer_count\x18\x03 \x01(\x05*/\n\tAlgorithm\x12\x10\n\x0cTOKEN_BUCKET\x10\x00\x12\x10\n\x0cLEAKY_BUCKET\x10\x01*e\n\x08\x42\x65havior\x12\x0c\n\x08\x42\x41TCHING\x10\x00\x12\x0f\n\x0bNO_BATCHING\x10\x01\x12\n\n\x06GLOBAL\x10\x02\x12\x19\n\x15\x44URATION_IS_GREGORIAN\x10\x04\x12\x13\n\x0fRESET_REMAINING\x10\x08*)\n\x06Status\x12\x0f\n\x0bUNDER_LIMIT\x10\x00\x12\x0e\n\nOVER_LIMIT\x10\x01\x32\xdd\x01\n\x02V1\x12p\n\rGetRateLimits\x12\x1f.pb.gubernator.GetRateLimitsReq\x1a .pb.gubernator.GetRateLimitsResp\"\x1c\x82\xd3\xe4\x93\x02\x16\"\x11/v1/GetRateLimits:\x01*\x12\x65\n\x0bHealthCheck\x12\x1d.pb.gubernator.HealthCheckReq\x1a\x1e.pb.gubernator.HealthCheckResp\"\x17\x82\xd3\xe4\x93\x02\x11\x12\x0f/v1/HealthCheckB\x0fZ\ngubernator\x80\x01\x01\x62\x06proto3') + serialized_pb=_b('\n\x10gubernator.proto\x12\rpb.gubernator\x1a\x1cgoogle/api/annotations.proto\"A\n\x10GetRateLimitsReq\x12-\n\x08requests\x18\x01 \x03(\x0b\x32\x1b.pb.gubernator.RateLimitReq\"D\n\x11GetRateLimitsResp\x12/\n\tresponses\x18\x01 \x03(\x0b\x32\x1c.pb.gubernator.RateLimitResp\"\xb7\x01\n\x0cRateLimitReq\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x12\n\nunique_key\x18\x02 \x01(\t\x12\x0c\n\x04hits\x18\x03 \x01(\x03\x12\r\n\x05limit\x18\x04 \x01(\x03\x12\x10\n\x08\x64uration\x18\x05 \x01(\x03\x12+\n\talgorithm\x18\x06 \x01(\x0e\x32\x18.pb.gubernator.Algorithm\x12)\n\x08\x62\x65havior\x18\x07 \x01(\x0e\x32\x17.pb.gubernator.Behavior\"\xea\x01\n\rRateLimitResp\x12%\n\x06status\x18\x01 \x01(\x0e\x32\x15.pb.gubernator.Status\x12\r\n\x05limit\x18\x02 \x01(\x03\x12\x11\n\tremaining\x18\x03 \x01(\x03\x12\x12\n\nreset_time\x18\x04 \x01(\x03\x12\r\n\x05\x65rror\x18\x05 \x01(\t\x12<\n\x08metadata\x18\x06 \x03(\x0b\x32*.pb.gubernator.RateLimitResp.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x10\n\x0eHealthCheckReq\"F\n\x0fHealthCheckResp\x12\x0e\n\x06status\x18\x01 \x01(\t\x12\x0f\n\x07message\x18\x02 \x01(\t\x12\x12\n\npeer_count\x18\x03 \x01(\x05*/\n\tAlgorithm\x12\x10\n\x0cTOKEN_BUCKET\x10\x00\x12\x10\n\x0cLEAKY_BUCKET\x10\x01*w\n\x08\x42\x65havior\x12\x0c\n\x08\x42\x41TCHING\x10\x00\x12\x0f\n\x0bNO_BATCHING\x10\x01\x12\n\n\x06GLOBAL\x10\x02\x12\x19\n\x15\x44URATION_IS_GREGORIAN\x10\x04\x12\x13\n\x0fRESET_REMAINING\x10\x08\x12\x10\n\x0cMULTI_REGION\x10\x10*)\n\x06Status\x12\x0f\n\x0bUNDER_LIMIT\x10\x00\x12\x0e\n\nOVER_LIMIT\x10\x01\x32\xdd\x01\n\x02V1\x12p\n\rGetRateLimits\x12\x1f.pb.gubernator.GetRateLimitsReq\x1a .pb.gubernator.GetRateLimitsResp\"\x1c\x82\xd3\xe4\x93\x02\x16\"\x11/v1/GetRateLimits:\x01*\x12\x65\n\x0bHealthCheck\x12\x1d.pb.gubernator.HealthCheckReq\x1a\x1e.pb.gubernator.HealthCheckResp\"\x17\x82\xd3\xe4\x93\x02\x11\x12\x0f/v1/HealthCheckB\x0fZ\ngubernator\x80\x01\x01\x62\x06proto3') , dependencies=[google_dot_api_dot_annotations__pb2.DESCRIPTOR,]) @@ -74,11 +75,15 @@ name='RESET_REMAINING', index=4, number=8, serialized_options=None, type=None), + _descriptor.EnumValueDescriptor( + name='MULTI_REGION', index=5, number=16, + serialized_options=None, + type=None), ], containing_type=None, serialized_options=None, serialized_start=764, - serialized_end=865, + serialized_end=883, ) _sym_db.RegisterEnumDescriptor(_BEHAVIOR) @@ -100,8 +105,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=867, - serialized_end=908, + serialized_start=885, + serialized_end=926, ) _sym_db.RegisterEnumDescriptor(_STATUS) @@ -113,6 +118,7 @@ GLOBAL = 2 DURATION_IS_GREGORIAN = 4 RESET_REMAINING = 8 +MULTI_REGION = 16 UNDER_LIMIT = 0 OVER_LIMIT = 1 @@ -442,54 +448,54 @@ DESCRIPTOR.enum_types_by_name['Status'] = _STATUS _sym_db.RegisterFileDescriptor(DESCRIPTOR) -GetRateLimitsReq = _reflection.GeneratedProtocolMessageType('GetRateLimitsReq', (_message.Message,), dict( - DESCRIPTOR = _GETRATELIMITSREQ, - __module__ = 'gubernator_pb2' +GetRateLimitsReq = _reflection.GeneratedProtocolMessageType('GetRateLimitsReq', (_message.Message,), { + 'DESCRIPTOR' : _GETRATELIMITSREQ, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.GetRateLimitsReq) - )) + }) _sym_db.RegisterMessage(GetRateLimitsReq) -GetRateLimitsResp = _reflection.GeneratedProtocolMessageType('GetRateLimitsResp', (_message.Message,), dict( - DESCRIPTOR = _GETRATELIMITSRESP, - __module__ = 'gubernator_pb2' +GetRateLimitsResp = _reflection.GeneratedProtocolMessageType('GetRateLimitsResp', (_message.Message,), { + 'DESCRIPTOR' : _GETRATELIMITSRESP, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.GetRateLimitsResp) - )) + }) _sym_db.RegisterMessage(GetRateLimitsResp) -RateLimitReq = _reflection.GeneratedProtocolMessageType('RateLimitReq', (_message.Message,), dict( - DESCRIPTOR = _RATELIMITREQ, - __module__ = 'gubernator_pb2' +RateLimitReq = _reflection.GeneratedProtocolMessageType('RateLimitReq', (_message.Message,), { + 'DESCRIPTOR' : _RATELIMITREQ, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.RateLimitReq) - )) + }) _sym_db.RegisterMessage(RateLimitReq) -RateLimitResp = _reflection.GeneratedProtocolMessageType('RateLimitResp', (_message.Message,), dict( +RateLimitResp = _reflection.GeneratedProtocolMessageType('RateLimitResp', (_message.Message,), { - MetadataEntry = _reflection.GeneratedProtocolMessageType('MetadataEntry', (_message.Message,), dict( - DESCRIPTOR = _RATELIMITRESP_METADATAENTRY, - __module__ = 'gubernator_pb2' + 'MetadataEntry' : _reflection.GeneratedProtocolMessageType('MetadataEntry', (_message.Message,), { + 'DESCRIPTOR' : _RATELIMITRESP_METADATAENTRY, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.RateLimitResp.MetadataEntry) - )) + }) , - DESCRIPTOR = _RATELIMITRESP, - __module__ = 'gubernator_pb2' + 'DESCRIPTOR' : _RATELIMITRESP, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.RateLimitResp) - )) + }) _sym_db.RegisterMessage(RateLimitResp) _sym_db.RegisterMessage(RateLimitResp.MetadataEntry) -HealthCheckReq = _reflection.GeneratedProtocolMessageType('HealthCheckReq', (_message.Message,), dict( - DESCRIPTOR = _HEALTHCHECKREQ, - __module__ = 'gubernator_pb2' +HealthCheckReq = _reflection.GeneratedProtocolMessageType('HealthCheckReq', (_message.Message,), { + 'DESCRIPTOR' : _HEALTHCHECKREQ, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.HealthCheckReq) - )) + }) _sym_db.RegisterMessage(HealthCheckReq) -HealthCheckResp = _reflection.GeneratedProtocolMessageType('HealthCheckResp', (_message.Message,), dict( - DESCRIPTOR = _HEALTHCHECKRESP, - __module__ = 'gubernator_pb2' +HealthCheckResp = _reflection.GeneratedProtocolMessageType('HealthCheckResp', (_message.Message,), { + 'DESCRIPTOR' : _HEALTHCHECKRESP, + '__module__' : 'gubernator_pb2' # @@protoc_insertion_point(class_scope:pb.gubernator.HealthCheckResp) - )) + }) _sym_db.RegisterMessage(HealthCheckResp) @@ -502,8 +508,8 @@ file=DESCRIPTOR, index=0, serialized_options=None, - serialized_start=911, - serialized_end=1132, + serialized_start=929, + serialized_end=1150, methods=[ _descriptor.MethodDescriptor( name='GetRateLimits', diff --git a/region_picker.go b/region_picker.go new file mode 100644 index 00000000..29dda7bf --- /dev/null +++ b/region_picker.go @@ -0,0 +1,95 @@ +package gubernator + +import ( + "github.com/mailgun/holster/v3/syncutil" +) + +type RegionPeerPicker interface { + GetClients(string) ([]*PeerClient, error) + GetByPeerInfo(PeerInfo) *PeerClient + Pickers() map[string]PeerPicker + Peers() []*PeerClient + Add(*PeerClient) + New() RegionPeerPicker +} + +// RegionPicker encapsulates pickers for a set of regions +type RegionPicker struct { + *ConsistantHash + + // A map of all the pickers by region + regions map[string]PeerPicker + // The implementation of picker we will use for each region + conf BehaviorConfig + wg syncutil.WaitGroup + reqQueue chan *RateLimitReq +} + +func NewRegionPicker(fn HashFunc) *RegionPicker { + rp := &RegionPicker{ + regions: make(map[string]PeerPicker), + reqQueue: make(chan *RateLimitReq, 0), + ConsistantHash: NewConsistantHash(fn), + } + return rp +} + +func (rp *RegionPicker) New() RegionPeerPicker { + hash := rp.ConsistantHash.New().(*ConsistantHash) + return &RegionPicker{ + regions: make(map[string]PeerPicker), + reqQueue: make(chan *RateLimitReq, 0), + ConsistantHash: hash, + } +} + +// GetClients returns all the PeerClients that match this key in all regions +func (rp *RegionPicker) GetClients(key string) ([]*PeerClient, error) { + result := make([]*PeerClient, len(rp.regions)) + var i int + for _, picker := range rp.regions { + peer, err := picker.Get(key) + if err != nil { + return nil, err + } + result[i] = peer + i++ + } + return result, nil +} + +// GetByPeerInfo returns the first PeerClient the PeerInfo.HasKey() matches +func (rp *RegionPicker) GetByPeerInfo(info PeerInfo) *PeerClient { + for _, picker := range rp.regions { + if client := picker.GetByPeerInfo(info); client != nil { + return client + } + } + return nil +} + +// Pickers returns a map of each region and its respective PeerPicker +func (rp *RegionPicker) Pickers() map[string]PeerPicker { + return rp.regions +} + +func (rp *RegionPicker) Peers() []*PeerClient { + var peers []*PeerClient + + for _, picker := range rp.regions { + for _, peer := range picker.Peers() { + peers = append(peers, peer) + } + } + + return peers +} + +func (rp *RegionPicker) Add(peer *PeerClient) { + picker, ok := rp.regions[peer.info.DataCenter] + if !ok { + picker = rp.ConsistantHash.New() + rp.regions[peer.info.DataCenter] = picker + } + picker.Add(peer) +} diff --git a/replicated_hash.go b/replicated_hash.go index 8d2256e1..9a1b3b7e 100644 --- a/replicated_hash.go +++ b/replicated_hash.go @@ -74,10 +74,10 @@ func (ch *ReplicatedConsistantHash) Peers() []*PeerClient { // Adds a peer to the hash func (ch *ReplicatedConsistantHash) Add(peer *PeerClient) { - ch.peers[peer.host] = peer + ch.peers[peer.info.Address] = peer for i := 0; i < ch.replicas; i++ { - hash := ch.hashFunc(bytes(strconv.Itoa(i) + peer.host)) + hash := ch.hashFunc(strToBytesUnsafe(strconv.Itoa(i) + peer.info.Address)) ch.peerKeys = append(ch.peerKeys, peerInfo{ hash: hash, peer: peer, @@ -93,8 +93,8 @@ func (ch *ReplicatedConsistantHash) Size() int { } // Returns the peer by hostname -func (ch *ReplicatedConsistantHash) GetPeerByHost(host string) *PeerClient { - return ch.peers[host] +func (ch *ReplicatedConsistantHash) GetByPeerInfo(peer PeerInfo) *PeerClient { + return ch.peers[peer.Address] } // Given a key, return the peer that key is assigned too @@ -102,7 +102,7 @@ func (ch *ReplicatedConsistantHash) Get(key string) (*PeerClient, error) { if ch.Size() == 0 { return nil, errors.New("unable to pick a peer; pool is empty") } - hash := ch.hashFunc(bytes(key)) + hash := ch.hashFunc(strToBytesUnsafe(key)) // Binary search for appropriate peer idx := sort.Search(len(ch.peerKeys), func(i int) bool { return ch.peerKeys[i].hash >= hash }) diff --git a/replicated_hash_test.go b/replicated_hash_test.go index b5ae2f14..cb2fd5e9 100644 --- a/replicated_hash_test.go +++ b/replicated_hash_test.go @@ -18,7 +18,7 @@ func TestReplicatedConsistantHash(t *testing.T) { hash := NewReplicatedConsistantHash(nil, DefaultReplicas) for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) } assert.Equal(t, len(hosts), hash.Size()) @@ -29,13 +29,13 @@ func TestReplicatedConsistantHash(t *testing.T) { hostMap := map[string]*PeerClient{} for _, h := range hosts { - peer := &PeerClient{host: h} + peer := &PeerClient{info: PeerInfo{Address: h}} hash.Add(peer) hostMap[h] = peer } for host, peer := range hostMap { - assert.Equal(t, peer, hash.GetPeerByHost(host)) + assert.Equal(t, peer, hash.GetByPeerInfo(PeerInfo{Address: host})) } }) @@ -62,13 +62,13 @@ func TestReplicatedConsistantHash(t *testing.T) { hostMap := map[string]int{} for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) hostMap[h] = 0 } for i := range strings { peer, _ := hash.Get(strings[i]) - hostMap[peer.host]++ + hostMap[peer.info.Address]++ } for host, a := range hostMap { @@ -96,7 +96,7 @@ func BenchmarkReplicatedConsistantHash(b *testing.B) { hash := NewReplicatedConsistantHash(hashFunc, DefaultReplicas) hosts := []string{"a.svc.local", "b.svc.local", "c.svc.local"} for _, h := range hosts { - hash.Add(&PeerClient{host: h}) + hash.Add(&PeerClient{info: PeerInfo{Address: h}}) } b.ResetTimer() diff --git a/scripts/proto.sh b/scripts/proto.sh index 19cd315d..86f89745 100755 --- a/scripts/proto.sh +++ b/scripts/proto.sh @@ -24,7 +24,7 @@ set -x PROTO_DIR=proto GO_DIR=. PY_DIR=python/gubernator -GRPC_GATEWAY_DIR=$GOPATH/pkg/mod/github.com/grpc-ecosystem/grpc-gateway\@v1.7.0/third_party/googleapis +GRPC_GATEWAY_DIR=$GOPATH/pkg/mod/github.com/grpc-ecosystem/grpc-gateway\@v1.11.1/third_party/googleapis protoc -I=$PROTO_DIR \ -I=$GRPC_GATEWAY_DIR \