From d788adc60c1df274f15774b156e764d818cf3d3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 15 Jan 2020 15:04:12 +0100 Subject: [PATCH 01/29] Added codec ID, to be used between memberlist clients. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/distributor/distributor_test.go | 3 +-- pkg/distributor/ha_tracker.go | 6 +++++- pkg/distributor/ha_tracker_test.go | 9 ++++---- pkg/ring/kv/codec/codec.go | 21 +++++++++++++++++-- pkg/ring/kv/memberlist/memberlist_client.go | 3 +++ .../kv/memberlist/memberlist_client_test.go | 4 ++++ pkg/ring/model.go | 2 +- pkg/ring/ring.go | 3 +-- pkg/ruler/ruler_test.go | 4 +--- 9 files changed, 39 insertions(+), 16 deletions(-) diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index d2cb0667df6..ba116b1513e 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -30,7 +30,6 @@ import ( "github.com/cortexproject/cortex/pkg/prom1/storage/metric" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/chunkcompat" @@ -250,7 +249,7 @@ func TestDistributor_PushHAInstances(t *testing.T) { limits.AcceptHASamples = true d, _ := prepare(t, 1, 1, 0, shardByAllLabels, &limits, nil) - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() mock := kv.PrefixClient(consul.NewInMemoryClient(codec), "prefix") if tc.enableTracker { diff --git a/pkg/distributor/ha_tracker.go b/pkg/distributor/ha_tracker.go index f5126ebdea1..f90ab4e0d49 100644 --- a/pkg/distributor/ha_tracker.go +++ b/pkg/distributor/ha_tracker.go @@ -136,10 +136,14 @@ func (cfg *HATrackerConfig) Validate() error { return nil } +func GetReplicaDescCodec() codec.Proto { + return codec.NewProtoCodec("replicaDesc", ProtoReplicaDescFactory) +} + // NewClusterTracker returns a new HA cluster tracker using either Consul // or in-memory KV store. func newClusterTracker(cfg HATrackerConfig) (*haTracker, error) { - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() var jitter time.Duration if cfg.UpdateTimeoutJitterMax > 0 { diff --git a/pkg/distributor/ha_tracker_test.go b/pkg/distributor/ha_tracker_test.go index 48e49dfc822..664dfa89373 100644 --- a/pkg/distributor/ha_tracker_test.go +++ b/pkg/distributor/ha_tracker_test.go @@ -15,7 +15,6 @@ import ( "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/util/flagext" ) @@ -127,7 +126,7 @@ func TestWatchPrefixAssignment(t *testing.T) { replica := "r1" start := mtime.Now() - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() mock := kv.PrefixClient(consul.NewInMemoryClient(codec), "prefix") c, err := newClusterTracker(HATrackerConfig{ EnableHATracker: true, @@ -269,7 +268,7 @@ func TestCheckReplicaUpdateTimeout(t *testing.T) { cluster := "c1" user := "user" - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() mock := kv.PrefixClient(consul.NewInMemoryClient(codec), "prefix") c, err := newClusterTracker(HATrackerConfig{ EnableHATracker: true, @@ -328,7 +327,7 @@ func TestCheckReplicaMultiUser(t *testing.T) { cluster := "c1" user := "user" - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() mock := kv.PrefixClient(consul.NewInMemoryClient(codec), "prefix") c, err := newClusterTracker(HATrackerConfig{ EnableHATracker: true, @@ -409,7 +408,7 @@ func TestCheckReplicaUpdateTimeoutJitter(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { // Init HA tracker - codec := codec.Proto{Factory: ProtoReplicaDescFactory} + codec := GetReplicaDescCodec() mock := kv.PrefixClient(consul.NewInMemoryClient(codec), "prefix") c, err := newClusterTracker(HATrackerConfig{ EnableHATracker: true, diff --git a/pkg/ring/kv/codec/codec.go b/pkg/ring/kv/codec/codec.go index f375a6d30b4..3aab3c9afd3 100644 --- a/pkg/ring/kv/codec/codec.go +++ b/pkg/ring/kv/codec/codec.go @@ -9,16 +9,29 @@ import ( type Codec interface { Decode([]byte) (interface{}, error) Encode(interface{}) ([]byte, error) + + // Short ID to communicate what codec should be used to decode the value. + // Once in use, this should be stable to avoid confusing other clients. + CodecID() string } // Proto is a Codec for proto/snappy type Proto struct { - Factory func() proto.Message + id string + factory func() proto.Message +} + +func NewProtoCodec(id string, factory func() proto.Message) Proto { + return Proto{id: id, factory: factory} +} + +func (p Proto) CodecID() string { + return p.id } // Decode implements Codec func (p Proto) Decode(bytes []byte) (interface{}, error) { - out := p.Factory() + out := p.factory() bytes, err := snappy.Decode(nil, bytes) if err != nil { return nil, err @@ -41,6 +54,10 @@ func (p Proto) Encode(msg interface{}) ([]byte, error) { // String is a code for strings. type String struct{} +func (String) CodecID() string { + return "string" +} + // Decode implements Codec. func (String) Decode(bytes []byte) (interface{}, error) { return string(bytes), nil diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 6958f9151ab..793562e13ff 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -124,6 +124,9 @@ type valueDesc struct { // version (local only) is used to keep track of what we're gossiping about, and invalidate old messages version uint + + // Codec used to write this value. + codecID string } var ( diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index fa19a800402..8bc8b99491c 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -97,6 +97,10 @@ func (d *data) getAllTokens() []uint32 { type dataCodec struct{} +func (d dataCodec) CodecID() string { + return "testDataCodec" +} + func (d dataCodec) Decode(b []byte) (interface{}, error) { dec := gob.NewDecoder(bytes.NewBuffer(b)) out := &data{} diff --git a/pkg/ring/model.go b/pkg/ring/model.go index 77a025513e3..3170deb9128 100644 --- a/pkg/ring/model.go +++ b/pkg/ring/model.go @@ -25,7 +25,7 @@ func ProtoDescFactory() proto.Message { // GetCodec returns the codec used to encode and decode data being put by ring. func GetCodec() codec.Codec { - return codec.Proto{Factory: ProtoDescFactory} + return codec.NewProtoCodec("ringDesc", ProtoDescFactory) } // NewDesc returns an empty ring.Desc diff --git a/pkg/ring/ring.go b/pkg/ring/ring.go index b944506f579..19f3f0283a9 100644 --- a/pkg/ring/ring.go +++ b/pkg/ring/ring.go @@ -16,7 +16,6 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/cortexproject/cortex/pkg/ring/kv" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/util" ) @@ -114,7 +113,7 @@ func New(cfg Config, name, key string) (*Ring, error) { if cfg.ReplicationFactor <= 0 { return nil, fmt.Errorf("ReplicationFactor must be greater than zero: %d", cfg.ReplicationFactor) } - codec := codec.Proto{Factory: ProtoDescFactory} + codec := GetCodec() store, err := kv.NewClient(cfg.KVStore, codec) if err != nil { return nil, err diff --git a/pkg/ruler/ruler_test.go b/pkg/ruler/ruler_test.go index c58469db8b7..3ff61f1b527 100644 --- a/pkg/ruler/ruler_test.go +++ b/pkg/ruler/ruler_test.go @@ -24,7 +24,6 @@ import ( "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/ring" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/ruler/rules" "github.com/cortexproject/cortex/pkg/util/flagext" @@ -34,8 +33,7 @@ func defaultRulerConfig(store rules.RuleStore) (Config, func()) { // Create a new temporary directory for the rules, so that // each test will run in isolation. rulesDir, _ := ioutil.TempDir("/tmp", "ruler-tests") - - codec := codec.Proto{Factory: ring.ProtoDescFactory} + codec := ring.GetCodec() consul := consul.NewInMemoryClient(codec) cfg := Config{} flagext.DefaultValues(&cfg) From 8e4e8d83a16d81e2d63c6d31f64aa51a61bd65c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Mon, 20 Jan 2020 13:55:36 +0100 Subject: [PATCH 02/29] Separated memberlist.Client from memberlist.KV. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This allows multiple clients to use the same memberlist.KV instance. This is another step towards supporting multiple codecs per KV. Signed-off-by: Peter Štibraný --- pkg/ring/kv/client.go | 22 +++-- pkg/ring/kv/memberlist/memberlist_client.go | 85 ++++++++++++------- .../kv/memberlist/memberlist_client_test.go | 32 ++++--- pkg/ring/kv/memberlist/metrics.go | 6 +- 4 files changed, 94 insertions(+), 51 deletions(-) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 9be4b8128ed..54737d8ffe1 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -20,14 +20,18 @@ import ( var inmemoryStoreInit sync.Once var inmemoryStore Client +var singletonKVStoreInit sync.Once +var singletonKVStore *memberlist.KV +var singletonKVStoreError error + // StoreConfig is a configuration used for building single store client, either // Consul, Etcd, Memberlist or MultiClient. It was extracted from Config to keep // single-client config separate from final client-config (with all the wrappers) type StoreConfig struct { - Consul consul.Config `yaml:"consul,omitempty"` - Etcd etcd.Config `yaml:"etcd,omitempty"` - Memberlist memberlist.Config `yaml:"memberlist,omitempty"` - Multi MultiConfig `yaml:"multi,omitempty"` + Consul consul.Config `yaml:"consul,omitempty"` + Etcd etcd.Config `yaml:"etcd,omitempty"` + Memberlist memberlist.KVConfig `yaml:"memberlist,omitempty"` + Multi MultiConfig `yaml:"multi,omitempty"` } // Config is config for a KVStore currently used by ring and HA tracker, @@ -120,7 +124,15 @@ func createClient(name string, prefix string, cfg StoreConfig, codec codec.Codec case "memberlist": cfg.Memberlist.MetricsRegisterer = prometheus.DefaultRegisterer - client, err = memberlist.NewMemberlistClient(cfg.Memberlist, codec) + singletonKVStoreInit.Do(func() { + singletonKVStore, singletonKVStoreError = memberlist.NewKV(cfg.Memberlist, codec) + }) + + kv, err := singletonKVStore, singletonKVStoreError + if err != nil { + return nil, err + } + client = memberlist.NewClient(kv) case "multi": client, err = buildMultiClient(cfg, codec) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 793562e13ff..9e345544532 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -27,8 +27,39 @@ const ( noChangeDetectedRetrySleep = time.Second // how long to sleep after no change was detected in CAS ) +type Client struct { + kv *KV // reference to singleton memberlist-based KV +} + +func NewClient(kv *KV) *Client { + return &Client{ + kv: kv, + } +} + +func (c *Client) Get(ctx context.Context, key string) (interface{}, error) { + return c.kv.Get(ctx, key) +} + +func (c *Client) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { + return c.kv.CAS(ctx, key, f) +} + +func (c *Client) WatchKey(ctx context.Context, key string, f func(interface{}) bool) { + c.kv.WatchKey(ctx, key, f) +} + +// WatchPrefix calls f whenever any value stored under prefix changes. +func (c *Client) WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) { + c.kv.WatchPrefix(ctx, prefix, f) +} + +func (c *Client) Stop() { + c.kv.Stop() +} + // Config for memberlist-based Client -type Config struct { +type KVConfig struct { // Memberlist options. NodeName string `yaml:"node_name"` StreamTimeout time.Duration `yaml:"stream_timeout"` @@ -55,7 +86,7 @@ type Config struct { } // RegisterFlags registers flags. -func (cfg *Config) RegisterFlags(f *flag.FlagSet, prefix string) { +func (cfg *KVConfig) RegisterFlags(f *flag.FlagSet, prefix string) { // "Defaults to hostname" -- memberlist sets it to hostname by default. f.StringVar(&cfg.NodeName, prefix+"memberlist.nodename", "", "Name of the node in memberlist cluster. Defaults to hostname.") // memberlist.DefaultLANConfig will put hostname here. f.DurationVar(&cfg.StreamTimeout, prefix+"memberlist.stream-timeout", 0, "The timeout for establishing a connection with a remote node, and for read/write operations. Uses memberlist LAN defaults if 0.") @@ -71,9 +102,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet, prefix string) { cfg.TCPTransport.RegisterFlags(f, prefix) } -// Client implements kv.Client interface by using memberlist gossiping library. -type Client struct { - cfg Config +type KV struct { + cfg KVConfig memberlist *memberlist.Memberlist broadcasts *memberlist.TransmitLimitedQueue codec codec.Codec @@ -124,9 +154,6 @@ type valueDesc struct { // version (local only) is used to keep track of what we're gossiping about, and invalidate old messages version uint - - // Codec used to write this value. - codecID string } var ( @@ -139,7 +166,7 @@ var ( // NewMemberlistClient creates new Client instance. If cfg.JoinMembers is set, it will also try to connect // to these members and join the cluster. If that fails and AbortIfJoinFails is true, error is returned and no // client is created. -func NewMemberlistClient(cfg Config, codec codec.Codec) (*Client, error) { +func NewKV(cfg KVConfig, codec codec.Codec) (*KV, error) { level.Warn(util.Logger).Log("msg", "Using memberlist-based KV store is EXPERIMENTAL and not tested in production") cfg.TCPTransport.MetricsRegisterer = cfg.MetricsRegisterer @@ -179,7 +206,7 @@ func NewMemberlistClient(cfg Config, codec codec.Codec) (*Client, error) { // As we don't use UDP for sending packets, we can use higher value here. mlCfg.UDPBufferSize = 10 * 1024 * 1024 - memberlistClient := &Client{ + memberlistClient := &KV{ cfg: cfg, store: make(map[string]valueDesc), watchers: make(map[string][]chan string), @@ -225,19 +252,19 @@ func NewMemberlistClient(cfg Config, codec codec.Codec) (*Client, error) { } // GetListeningPort returns port used for listening for memberlist communication. Useful when BindPort is set to 0. -func (m *Client) GetListeningPort() int { +func (m *KV) GetListeningPort() int { return int(m.memberlist.LocalNode().Port) } // JoinMembers joins the cluster with given members. // See https://godoc.org/github.com/hashicorp/memberlist#Memberlist.Join -func (m *Client) JoinMembers(members []string) (int, error) { +func (m *KV) JoinMembers(members []string) (int, error) { return m.memberlist.Join(members) } // Stop tries to leave memberlist cluster and then shutdown memberlist client. // We do this in order to send out last messages, typically that ingester has LEFT the ring. -func (m *Client) Stop() { +func (m *KV) Stop() { level.Info(util.Logger).Log("msg", "leaving memberlist cluster") // TODO: should we empty our broadcast queue before leaving? That would make sure that we have sent out everything we wanted. @@ -257,13 +284,13 @@ func (m *Client) Stop() { // Get returns current value associated with given key. // No communication with other nodes in the cluster is done here. Part of kv.Client interface. -func (m *Client) Get(ctx context.Context, key string) (interface{}, error) { +func (m *KV) Get(ctx context.Context, key string) (interface{}, error) { val, _, err := m.get(key) return val, err } // Returns current value with removed tombstones. -func (m *Client) get(key string) (out interface{}, version uint, err error) { +func (m *KV) get(key string) (out interface{}, version uint, err error) { m.storeMu.Lock() v := m.store[key] m.storeMu.Unlock() @@ -291,7 +318,7 @@ func (m *Client) get(key string) (out interface{}, version uint, err error) { // Watching ends when 'f' returns false, context is done, or this client is shut down. // // Part of kv.Client interface. -func (m *Client) WatchKey(ctx context.Context, key string, f func(interface{}) bool) { +func (m *KV) WatchKey(ctx context.Context, key string, f func(interface{}) bool) { // keep one extra notification, to avoid missing notification if we're busy running the function w := make(chan string, 1) @@ -340,7 +367,7 @@ func (m *Client) WatchKey(ctx context.Context, key string, f func(interface{}) b // Watching ends when 'f' returns false, context is done, or this client is shut down. // // Part of kv.Client interface. -func (m *Client) WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) { +func (m *KV) WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) { // we use bigger buffer here, since keys are interesting and we don't want to lose them. w := make(chan string, 16) @@ -396,7 +423,7 @@ func removeWatcherChannel(k string, w chan string, watchers map[string][]chan st } } -func (m *Client) notifyWatchers(key string) { +func (m *KV) notifyWatchers(key string) { m.watchersMu.Lock() defer m.watchersMu.Unlock() @@ -439,7 +466,7 @@ func (m *Client) notifyWatchers(key string) { // KV store, and change is broadcast to cluster peers. Merge function is called with CAS flag on, so that it can // detect removals. If Merge doesn't result in any change (returns nil), then operation fails and is retried again. // After too many failed retries, this method returns error. -func (m *Client) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { +func (m *KV) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { if len(key) > maxKeyLength { return fmt.Errorf("key too long: %d", len(key)) } @@ -495,7 +522,7 @@ outer: // returns change, error (or nil, if CAS succeeded), and whether to retry or not. // returns errNoChangeDetected if merge failed to detect change in f's output. -func (m *Client) trySingleCas(key string, f func(in interface{}) (out interface{}, retry bool, err error)) (Mergeable, uint, bool, error) { +func (m *KV) trySingleCas(key string, f func(in interface{}) (out interface{}, retry bool, err error)) (Mergeable, uint, bool, error) { val, ver, err := m.get(key) if err != nil { return nil, 0, false, fmt.Errorf("failed to get value: %v", err) @@ -536,7 +563,7 @@ func (m *Client) trySingleCas(key string, f func(in interface{}) (out interface{ return change, newver, retry, nil } -func (m *Client) broadcastNewValue(key string, change Mergeable, version uint) { +func (m *KV) broadcastNewValue(key string, change Mergeable, version uint) { data, err := m.codec.Encode(change) if err != nil { level.Error(util.Logger).Log("msg", "failed to encode ring", "err", err) @@ -563,7 +590,7 @@ func (m *Client) broadcastNewValue(key string, change Mergeable, version uint) { } // NodeMeta is method from Memberlist Delegate interface -func (m *Client) NodeMeta(limit int) []byte { +func (m *KV) NodeMeta(limit int) []byte { // we can send local state from here (512 bytes only) // if state is updated, we need to tell memberlist to distribute it. return nil @@ -571,7 +598,7 @@ func (m *Client) NodeMeta(limit int) []byte { // NotifyMsg is method from Memberlist Delegate interface // Called when single message is received, i.e. what our broadcastNewValue has sent. -func (m *Client) NotifyMsg(msg []byte) { +func (m *KV) NotifyMsg(msg []byte) { m.numberOfReceivedMessages.Inc() m.totalSizeOfReceivedMessages.Add(float64(len(msg))) @@ -607,7 +634,7 @@ func (m *Client) NotifyMsg(msg []byte) { } } -func (m *Client) queueBroadcast(key string, content []string, version uint, message []byte) { +func (m *KV) queueBroadcast(key string, content []string, version uint, message []byte) { l := len(message) b := ringBroadcast{ @@ -626,7 +653,7 @@ func (m *Client) queueBroadcast(key string, content []string, version uint, mess // GetBroadcasts is method from Memberlist Delegate interface // It returns all pending broadcasts (within the size limit) -func (m *Client) GetBroadcasts(overhead, limit int) [][]byte { +func (m *KV) GetBroadcasts(overhead, limit int) [][]byte { return m.broadcasts.GetBroadcasts(overhead, limit) } @@ -635,7 +662,7 @@ func (m *Client) GetBroadcasts(overhead, limit int) [][]byte { // This is "pull" part of push/pull sync (either periodic, or when new node joins the cluster). // Here we dump our entire state -- all keys and their values. There is no limit on message size here, // as Memberlist uses 'stream' operations for transferring this state. -func (m *Client) LocalState(join bool) []byte { +func (m *KV) LocalState(join bool) []byte { m.numberOfPulls.Inc() m.storeMu.Lock() @@ -676,7 +703,7 @@ func (m *Client) LocalState(join bool) []byte { // MergeRemoteState is method from Memberlist Delegate interface // // This is 'push' part of push/pull sync. We merge incoming KV store (all keys and values) with ours. -func (m *Client) MergeRemoteState(stateMsg []byte, join bool) { +func (m *KV) MergeRemoteState(stateMsg []byte, join bool) { m.numberOfPushes.Inc() m.totalSizeOfPushes.Add(float64(len(stateMsg))) @@ -742,7 +769,7 @@ func (m *Client) MergeRemoteState(stateMsg []byte, join bool) { } } -func (m *Client) mergeBytesValueForKey(key string, incomingData []byte) (Mergeable, uint, error) { +func (m *KV) mergeBytesValueForKey(key string, incomingData []byte) (Mergeable, uint, error) { decodedValue, err := m.codec.Decode(incomingData) if err != nil { return nil, 0, fmt.Errorf("failed to decode value: %v", err) @@ -760,7 +787,7 @@ func (m *Client) mergeBytesValueForKey(key string, incomingData []byte) (Mergeab // cluster members to update their state, and new version of the value. // If CAS version is specified, then merging will fail if state has changed already, and errVersionMismatch is reported. // If no modification occurred, new version is 0. -func (m *Client) mergeValueForKey(key string, incomingValue Mergeable, casVersion uint) (Mergeable, uint, error) { +func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion uint) (Mergeable, uint, error) { m.storeMu.Lock() defer m.storeMu.Unlock() diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index 8bc8b99491c..f557d74a974 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -204,17 +204,18 @@ func TestBasicGetAndCas(t *testing.T) { c := dataCodec{} name := "Ing 1" - cfg := Config{ + cfg := KVConfig{ TCPTransport: TCPTransportConfig{ BindAddrs: []string{"localhost"}, }, } - kv, err := NewMemberlistClient(cfg, c) + mkv, err := NewKV(cfg, c) if err != nil { t.Fatal("Failed to setup KV client", err) } - defer kv.Stop() + defer mkv.Stop() + kv := NewClient(mkv) const key = "test" @@ -260,15 +261,16 @@ func withFixtures(t *testing.T, testFN func(t *testing.T, kv *Client)) { c := dataCodec{} - cfg := Config{ + cfg := KVConfig{ TCPTransport: TCPTransportConfig{}, } - kv, err := NewMemberlistClient(cfg, c) + mkv, err := NewKV(cfg, c) if err != nil { t.Fatal("Failed to setup KV client", err) } - defer kv.Stop() + defer mkv.Stop() + kv := NewClient(mkv) testFN(t, kv) } @@ -403,13 +405,14 @@ func TestCASFailedBecauseOfVersionChanges(t *testing.T) { func TestMultipleCAS(t *testing.T) { c := dataCodec{} - cfg := Config{} + cfg := KVConfig{} - kv, err := NewMemberlistClient(cfg, c) + mkv, err := NewKV(cfg, c) if err != nil { t.Fatal("Failed to setup KV client", err) } - kv.maxCasRetries = 20 + mkv.maxCasRetries = 20 + kv := NewClient(mkv) defer kv.Stop() wg := &sync.WaitGroup{} @@ -493,7 +496,7 @@ func TestMultipleClients(t *testing.T) { for i := 0; i < members; i++ { id := fmt.Sprintf("Member-%d", i) - cfg := Config{ + cfg := KVConfig{ NodeName: id, // some useful parameters when playing with higher number of members @@ -511,17 +514,18 @@ func TestMultipleClients(t *testing.T) { }, } - kv, err := NewMemberlistClient(cfg, c) + mkv, err := NewKV(cfg, c) if err != nil { - t.Fatal(id, err) + t.Fatal(id, "Failed to setup KV client", err) } + kv := NewClient(mkv) clients = append(clients, kv) go runClient(t, kv, id, key, port, start, stop) // next KV will connect to this one - port = kv.GetListeningPort() + port = kv.kv.GetListeningPort() } println("Waiting before start") @@ -637,7 +641,7 @@ func runClient(t *testing.T, kv *Client, name string, ringKey string, portToConn // let's join the first member if portToConnect > 0 { - _, err := kv.JoinMembers([]string{fmt.Sprintf("127.0.0.1:%d", portToConnect)}) + _, err := kv.kv.JoinMembers([]string{fmt.Sprintf("127.0.0.1:%d", portToConnect)}) if err != nil { t.Fatalf("%s failed to join the cluster: %v", name, err) return diff --git a/pkg/ring/kv/memberlist/metrics.go b/pkg/ring/kv/memberlist/metrics.go index cde9571d862..b3a7fa504be 100644 --- a/pkg/ring/kv/memberlist/metrics.go +++ b/pkg/ring/kv/memberlist/metrics.go @@ -11,7 +11,7 @@ import ( "github.com/cortexproject/cortex/pkg/util" ) -func (m *Client) createAndRegisterMetrics() { +func (m *KV) createAndRegisterMetrics() { const subsystem = "memberlist_client" m.numberOfReceivedMessages = prometheus.NewCounter(prometheus.CounterOpts{ @@ -183,13 +183,13 @@ func (m *Client) createAndRegisterMetrics() { } // Describe returns prometheus descriptions via supplied channel -func (m *Client) Describe(ch chan<- *prometheus.Desc) { +func (m *KV) Describe(ch chan<- *prometheus.Desc) { ch <- m.storeValuesDesc ch <- m.storeSizesDesc } // Collect returns extra metrics via supplied channel -func (m *Client) Collect(ch chan<- prometheus.Metric) { +func (m *KV) Collect(ch chan<- prometheus.Metric) { m.storeMu.Lock() defer m.storeMu.Unlock() From 65b1a61293f6bde3aabea51eb34f670f315dd22c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Mon, 20 Jan 2020 15:54:19 +0100 Subject: [PATCH 03/29] Replaced custom serialization with protobuf serialization. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit To support multiple codecs, we need to add a new field. As we're going to break the format anyway, we may as well switch to protobuf. Signed-off-by: Peter Štibraný --- Makefile | 1 + pkg/ring/kv/memberlist/kv.pb.go | 694 ++++++++++++++++++++ pkg/ring/kv/memberlist/kv.proto | 19 + pkg/ring/kv/memberlist/memberlist_client.go | 127 ++-- 4 files changed, 767 insertions(+), 74 deletions(-) create mode 100644 pkg/ring/kv/memberlist/kv.pb.go create mode 100644 pkg/ring/kv/memberlist/kv.proto diff --git a/Makefile b/Makefile index 8cdb2199a06..136b82617c1 100644 --- a/Makefile +++ b/Makefile @@ -56,6 +56,7 @@ pkg/querier/queryrange/queryrange.pb.go: pkg/querier/queryrange/queryrange.proto pkg/chunk/storage/caching_index_client.pb.go: pkg/chunk/storage/caching_index_client.proto pkg/distributor/ha_tracker.pb.go: pkg/distributor/ha_tracker.proto pkg/ruler/rules/rules.pb.go: pkg/ruler/rules/rules.proto +pkg/ring/kv/memberlist/kv.pb.go: pkg/ring/kv/memberlist/kv.proto all: $(UPTODATE_FILES) test: protos diff --git a/pkg/ring/kv/memberlist/kv.pb.go b/pkg/ring/kv/memberlist/kv.pb.go new file mode 100644 index 00000000000..ef4a63a5800 --- /dev/null +++ b/pkg/ring/kv/memberlist/kv.pb.go @@ -0,0 +1,694 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: kv.proto + +package memberlist + +import ( + bytes "bytes" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + reflect "reflect" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +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.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// KV Store is just a series of key-value pairs. +type KeyValueStore struct { + Pairs []*KeyValuePair `protobuf:"bytes,1,rep,name=pairs,proto3" json:"pairs,omitempty"` +} + +func (m *KeyValueStore) Reset() { *m = KeyValueStore{} } +func (*KeyValueStore) ProtoMessage() {} +func (*KeyValueStore) Descriptor() ([]byte, []int) { + return fileDescriptor_2216fe83c9c12408, []int{0} +} +func (m *KeyValueStore) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *KeyValueStore) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_KeyValueStore.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *KeyValueStore) XXX_Merge(src proto.Message) { + xxx_messageInfo_KeyValueStore.Merge(m, src) +} +func (m *KeyValueStore) XXX_Size() int { + return m.Size() +} +func (m *KeyValueStore) XXX_DiscardUnknown() { + xxx_messageInfo_KeyValueStore.DiscardUnknown(m) +} + +var xxx_messageInfo_KeyValueStore proto.InternalMessageInfo + +func (m *KeyValueStore) GetPairs() []*KeyValuePair { + if m != nil { + return m.Pairs + } + return nil +} + +// Single Key-Value pair. Key must be non-empty. +type KeyValuePair struct { + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (m *KeyValuePair) Reset() { *m = KeyValuePair{} } +func (*KeyValuePair) ProtoMessage() {} +func (*KeyValuePair) Descriptor() ([]byte, []int) { + return fileDescriptor_2216fe83c9c12408, []int{1} +} +func (m *KeyValuePair) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *KeyValuePair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_KeyValuePair.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *KeyValuePair) XXX_Merge(src proto.Message) { + xxx_messageInfo_KeyValuePair.Merge(m, src) +} +func (m *KeyValuePair) XXX_Size() int { + return m.Size() +} +func (m *KeyValuePair) XXX_DiscardUnknown() { + xxx_messageInfo_KeyValuePair.DiscardUnknown(m) +} + +var xxx_messageInfo_KeyValuePair proto.InternalMessageInfo + +func (m *KeyValuePair) GetKey() string { + if m != nil { + return m.Key + } + return "" +} + +func (m *KeyValuePair) GetValue() []byte { + if m != nil { + return m.Value + } + return nil +} + +func init() { + proto.RegisterType((*KeyValueStore)(nil), "memberlist.KeyValueStore") + proto.RegisterType((*KeyValuePair)(nil), "memberlist.KeyValuePair") +} + +func init() { proto.RegisterFile("kv.proto", fileDescriptor_2216fe83c9c12408) } + +var fileDescriptor_2216fe83c9c12408 = []byte{ + // 223 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0xc8, 0x2e, 0xd3, 0x2b, + 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0xca, 0x4d, 0xcd, 0x4d, 0x4a, 0x2d, 0xca, 0xc9, 0x2c, 0x2e, + 0x91, 0xd2, 0x4d, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x4f, 0xcf, 0x4f, + 0xcf, 0xd7, 0x07, 0x2b, 0x49, 0x2a, 0x4d, 0x03, 0xf3, 0xc0, 0x1c, 0x30, 0x0b, 0xa2, 0x55, 0xc9, + 0x9e, 0x8b, 0xd7, 0x3b, 0xb5, 0x32, 0x2c, 0x31, 0xa7, 0x34, 0x35, 0xb8, 0x24, 0xbf, 0x28, 0x55, + 0x48, 0x8f, 0x8b, 0xb5, 0x20, 0x31, 0xb3, 0xa8, 0x58, 0x82, 0x51, 0x81, 0x59, 0x83, 0xdb, 0x48, + 0x42, 0x0f, 0x61, 0xb6, 0x1e, 0x4c, 0x65, 0x40, 0x62, 0x66, 0x51, 0x10, 0x44, 0x99, 0x92, 0x19, + 0x17, 0x0f, 0xb2, 0xb0, 0x90, 0x00, 0x17, 0x73, 0x76, 0x6a, 0xa5, 0x04, 0xa3, 0x02, 0xa3, 0x06, + 0x67, 0x10, 0x88, 0x29, 0x24, 0xc2, 0xc5, 0x5a, 0x06, 0x92, 0x96, 0x60, 0x52, 0x60, 0xd4, 0xe0, + 0x09, 0x82, 0x70, 0x9c, 0x4c, 0x2e, 0x3c, 0x94, 0x63, 0xb8, 0xf1, 0x50, 0x8e, 0xe1, 0xc3, 0x43, + 0x39, 0xc6, 0x86, 0x47, 0x72, 0x8c, 0x2b, 0x1e, 0xc9, 0x31, 0x9e, 0x78, 0x24, 0xc7, 0x78, 0xe1, + 0x91, 0x1c, 0xe3, 0x83, 0x47, 0x72, 0x8c, 0x2f, 0x1e, 0xc9, 0x31, 0x7c, 0x78, 0x24, 0xc7, 0x38, + 0xe1, 0xb1, 0x1c, 0xc3, 0x85, 0xc7, 0x72, 0x0c, 0x37, 0x1e, 0xcb, 0x31, 0x24, 0xb1, 0x81, 0x5d, + 0x6d, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x31, 0x79, 0x71, 0xc6, 0xfc, 0x00, 0x00, 0x00, +} + +func (this *KeyValueStore) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*KeyValueStore) + if !ok { + that2, ok := that.(KeyValueStore) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Pairs) != len(that1.Pairs) { + return false + } + for i := range this.Pairs { + if !this.Pairs[i].Equal(that1.Pairs[i]) { + return false + } + } + return true +} +func (this *KeyValuePair) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*KeyValuePair) + if !ok { + that2, ok := that.(KeyValuePair) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Key != that1.Key { + return false + } + if !bytes.Equal(this.Value, that1.Value) { + return false + } + return true +} +func (this *KeyValueStore) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&memberlist.KeyValueStore{") + if this.Pairs != nil { + s = append(s, "Pairs: "+fmt.Sprintf("%#v", this.Pairs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *KeyValuePair) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&memberlist.KeyValuePair{") + s = append(s, "Key: "+fmt.Sprintf("%#v", this.Key)+",\n") + s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringKv(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *KeyValueStore) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *KeyValueStore) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Pairs) > 0 { + for _, msg := range m.Pairs { + dAtA[i] = 0xa + i++ + i = encodeVarintKv(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *KeyValuePair) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *KeyValuePair) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Key) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintKv(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) + } + if len(m.Value) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKv(dAtA, i, uint64(len(m.Value))) + i += copy(dAtA[i:], m.Value) + } + return i, nil +} + +func encodeVarintKv(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *KeyValueStore) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Pairs) > 0 { + for _, e := range m.Pairs { + l = e.Size() + n += 1 + l + sovKv(uint64(l)) + } + } + return n +} + +func (m *KeyValuePair) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKv(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovKv(uint64(l)) + } + return n +} + +func sovKv(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozKv(x uint64) (n int) { + return sovKv(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *KeyValueStore) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&KeyValueStore{`, + `Pairs:` + strings.Replace(fmt.Sprintf("%v", this.Pairs), "KeyValuePair", "KeyValuePair", 1) + `,`, + `}`, + }, "") + return s +} +func (this *KeyValuePair) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&KeyValuePair{`, + `Key:` + fmt.Sprintf("%v", this.Key) + `,`, + `Value:` + fmt.Sprintf("%v", this.Value) + `,`, + `}`, + }, "") + return s +} +func valueToStringKv(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *KeyValueStore) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: KeyValueStore: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeyValueStore: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKv + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthKv + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pairs = append(m.Pairs, &KeyValuePair{}) + if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKv(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKv + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthKv + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *KeyValuePair) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: KeyValuePair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeyValuePair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKv + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthKv + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKv + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthKv + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKv(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKv + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthKv + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipKv(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowKv + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowKv + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowKv + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthKv + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthKv + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowKv + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipKv(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthKv + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthKv = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowKv = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/ring/kv/memberlist/kv.proto b/pkg/ring/kv/memberlist/kv.proto new file mode 100644 index 00000000000..1d2b578a99d --- /dev/null +++ b/pkg/ring/kv/memberlist/kv.proto @@ -0,0 +1,19 @@ +syntax = "proto3"; + +package memberlist; + +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option (gogoproto.marshaler_all) = true; +option (gogoproto.unmarshaler_all) = true; + +// KV Store is just a series of key-value pairs. +message KeyValueStore { + repeated KeyValuePair pairs = 1; +} + +// Single Key-Value pair. Key must be non-empty. +message KeyValuePair { + string key = 1; + bytes value = 2; +} diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 9e345544532..d75683e7d6e 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -12,17 +12,15 @@ import ( "sync" "time" - "github.com/go-kit/kit/log/level" - "github.com/hashicorp/memberlist" - "github.com/prometheus/client_golang/prometheus" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/go-kit/kit/log/level" + "github.com/hashicorp/memberlist" + "github.com/prometheus/client_golang/prometheus" ) const ( - maxKeyLength = 255 // We encode key length as one byte maxCasRetries = 10 // max retries in CAS operation noChangeDetectedRetrySleep = time.Second // how long to sleep after no change was detected in CAS ) @@ -467,10 +465,6 @@ func (m *KV) notifyWatchers(key string) { // detect removals. If Merge doesn't result in any change (returns nil), then operation fails and is retried again. // After too many failed retries, this method returns error. func (m *KV) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { - if len(key) > maxKeyLength { - return fmt.Errorf("key too long: %d", len(key)) - } - var lastError error = nil outer: @@ -566,27 +560,28 @@ func (m *KV) trySingleCas(key string, f func(in interface{}) (out interface{}, r func (m *KV) broadcastNewValue(key string, change Mergeable, version uint) { data, err := m.codec.Encode(change) if err != nil { - level.Error(util.Logger).Log("msg", "failed to encode ring", "err", err) + level.Error(util.Logger).Log("msg", "failed to encode change", "err", err) return } - buf := bytes.Buffer{} - buf.Write([]byte{byte(len(key))}) - buf.WriteString(key) - buf.Write(data) + kvPair := KeyValuePair{Key: key, Value: data} + pairData, err := kvPair.Marshal() + if err != nil { + level.Error(util.Logger).Log("msg", "failed to serialize KV pair", "err", err) + } - if buf.Len() > 65535 { + if len(pairData) > 65535 { // Unfortunately, memberlist will happily let us send bigger messages via gossip, // but then it will fail to parse them properly, because its own size field is 2-bytes only. // (github.com/hashicorp/memberlist@v0.1.4/util.go:167, makeCompoundMessage function) // // Typically messages are smaller (when dealing with couple of updates only), but can get bigger // when broadcasting result of push/pull update. - level.Debug(util.Logger).Log("msg", "broadcast message too big, not broadcasting", "len", buf.Len()) + level.Debug(util.Logger).Log("msg", "broadcast message too big, not broadcasting", "len", len(pairData)) return } - m.queueBroadcast(key, change.MergeContent(), version, buf.Bytes()) + m.queueBroadcast(key, change.MergeContent(), version, pairData) } // NodeMeta is method from Memberlist Delegate interface @@ -602,35 +597,33 @@ func (m *KV) NotifyMsg(msg []byte) { m.numberOfReceivedMessages.Inc() m.totalSizeOfReceivedMessages.Add(float64(len(msg))) - if len(msg) == 0 { - level.Warn(util.Logger).Log("msg", "Empty message received") + kvPair := KeyValuePair{} + err := kvPair.Unmarshal(msg) + if err != nil { + level.Warn(util.Logger).Log("msg", "Failed to unmarshal received KV Pair", "err", err) m.numberOfInvalidReceivedMessages.Inc() return } - keyLen := int(msg[0]) - if len(msg) <= 1+keyLen { - level.Warn(util.Logger).Log("msg", "Too short message received", "length", len(msg)) + if len(kvPair.Key) == 0 { + level.Warn(util.Logger).Log("msg", "Invalid KV Pair, empty key") m.numberOfInvalidReceivedMessages.Inc() return } - key := string(msg[1 : 1+keyLen]) - data := msg[1+keyLen:] - // we have a ring update! Let's merge it with our version of the ring for given key - mod, version, err := m.mergeBytesValueForKey(key, data) + mod, version, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value) if err != nil { - level.Error(util.Logger).Log("msg", "failed to store received value", "key", key, "err", err) + level.Error(util.Logger).Log("msg", "failed to store received value", "key", kvPair.Key, "err", err) } else if version > 0 { - m.notifyWatchers(key) + m.notifyWatchers(kvPair.Key) // Forward this message // Memberlist will modify message once this function returns, so we need to make a copy msgCopy := append([]byte(nil), msg...) // forward this message further - m.queueBroadcast(key, mod.MergeContent(), version, msgCopy) + m.queueBroadcast(kvPair.Key, mod.MergeContent(), version, msgCopy) } } @@ -671,29 +664,35 @@ func (m *KV) LocalState(join bool) []byte { // For each Key/Value pair in our store, we write: // [1 byte key length] [key] [4-bytes value length] [value] + kvPair := KeyValuePair{} + buf := bytes.Buffer{} for key, val := range m.store { if val.value == nil { continue } - if len(key) > maxKeyLength { - level.Error(util.Logger).Log("msg", "key too long", "key", key) + kvPair.Reset() + kvPair.Key = key + kvPair.Value = val.value + + ser, err := kvPair.Marshal() + if err != nil { + level.Error(util.Logger).Log("msg", "failed to serialize KV Pair", "err", err) continue } - if uint(len(val.value)) > math.MaxUint32 { + + if uint(len(ser)) > math.MaxUint32 { level.Error(util.Logger).Log("msg", "value too long", "key", key, "value_length", len(val.value)) continue } - buf.WriteByte(byte(len(key))) - buf.WriteString(key) - err := binary.Write(&buf, binary.BigEndian, uint32(len(val.value))) + err = binary.Write(&buf, binary.BigEndian, uint32(len(ser))) if err != nil { level.Error(util.Logger).Log("msg", "failed to write uint32 to buffer?", "err", err) continue } - buf.Write(val.value) + buf.Write(ser) } m.totalSizeOfPulls.Add(float64(buf.Len())) @@ -703,64 +702,44 @@ func (m *KV) LocalState(join bool) []byte { // MergeRemoteState is method from Memberlist Delegate interface // // This is 'push' part of push/pull sync. We merge incoming KV store (all keys and values) with ours. -func (m *KV) MergeRemoteState(stateMsg []byte, join bool) { +func (m *KV) MergeRemoteState(data []byte, join bool) { m.numberOfPushes.Inc() - m.totalSizeOfPushes.Add(float64(len(stateMsg))) + m.totalSizeOfPushes.Add(float64(len(data))) - buf := bytes.NewBuffer(stateMsg) + kvPair := KeyValuePair{} var err error - for buf.Len() > 0 { - keyLen := byte(0) - keyLen, err = buf.ReadByte() - if err != nil { - break - } - - keyBuf := make([]byte, keyLen) - l := 0 - l, err = buf.Read(keyBuf) - if err != nil { - break - } - - if l != len(keyBuf) { - err = fmt.Errorf("cannot read key, expected %d, got %d bytes", keyLen, l) + for len(data) > 0 { + if len(data) < 4 { + err = fmt.Errorf("not enough data left for another KV Pair: %d", len(data)) break } - key := string(keyBuf) + kvPairLength := binary.BigEndian.Uint32(data) - // next read the length of the data - valueLength := uint32(0) - err = binary.Read(buf, binary.BigEndian, &valueLength) - if err != nil { - break - } + data = data[4:] - if buf.Len() < int(valueLength) { - err = fmt.Errorf("not enough data left for value in key %q, expected %d, remaining %d bytes", key, valueLength, buf.Len()) + if len(data) < int(kvPairLength) { + err = fmt.Errorf("not enough data left for next KV Pair, expected %d, remaining %d bytes", kvPairLength, len(data)) break } - valueBuf := make([]byte, valueLength) - l, err = buf.Read(valueBuf) + kvPair.Reset() + err = kvPair.Unmarshal(data[:kvPairLength]) if err != nil { + err = fmt.Errorf("failed to parse KV Pair: %v", err) break } - if l != len(valueBuf) { - err = fmt.Errorf("cannot read value for key %q, expected %d, got %d bytes", key, valueLength, l) - break - } + data = data[kvPairLength:] // we have both key and value, try to merge it with our state - change, newver, err := m.mergeBytesValueForKey(key, valueBuf) + change, newver, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value) if err != nil { - level.Error(util.Logger).Log("msg", "failed to store received value", "key", key, "err", err) + level.Error(util.Logger).Log("msg", "failed to store received value", "key", kvPair.Key, "err", err) } else if newver > 0 { - m.notifyWatchers(key) - m.broadcastNewValue(key, change, newver) + m.notifyWatchers(kvPair.Key) + m.broadcastNewValue(kvPair.Key, change, newver) } } From d279c92f30fa62aa58d2f7ea55cd34d73a7c78bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 11:51:55 +0100 Subject: [PATCH 04/29] Codec ID is now part of the message exchanged between memberlist nodes. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Codec is now stored in the client, while KV has a registry of codecs that are used when receiving values. Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/kv.pb.go | 73 ++++++++-- pkg/ring/kv/memberlist/kv.proto | 3 + pkg/ring/kv/memberlist/memberlist_client.go | 133 ++++++++++++------ .../kv/memberlist/memberlist_client_test.go | 21 ++- 4 files changed, 167 insertions(+), 63 deletions(-) diff --git a/pkg/ring/kv/memberlist/kv.pb.go b/pkg/ring/kv/memberlist/kv.pb.go index ef4a63a5800..610a2f40f5c 100644 --- a/pkg/ring/kv/memberlist/kv.pb.go +++ b/pkg/ring/kv/memberlist/kv.pb.go @@ -73,6 +73,8 @@ func (m *KeyValueStore) GetPairs() []*KeyValuePair { type KeyValuePair struct { Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + // ID of the codec used to write the value + Codec string `protobuf:"bytes,3,opt,name=codec,proto3" json:"codec,omitempty"` } func (m *KeyValuePair) Reset() { *m = KeyValuePair{} } @@ -121,6 +123,13 @@ func (m *KeyValuePair) GetValue() []byte { return nil } +func (m *KeyValuePair) GetCodec() string { + if m != nil { + return m.Codec + } + return "" +} + func init() { proto.RegisterType((*KeyValueStore)(nil), "memberlist.KeyValueStore") proto.RegisterType((*KeyValuePair)(nil), "memberlist.KeyValuePair") @@ -129,21 +138,22 @@ func init() { func init() { proto.RegisterFile("kv.proto", fileDescriptor_2216fe83c9c12408) } var fileDescriptor_2216fe83c9c12408 = []byte{ - // 223 bytes of a gzipped FileDescriptorProto + // 236 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0xc8, 0x2e, 0xd3, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0xca, 0x4d, 0xcd, 0x4d, 0x4a, 0x2d, 0xca, 0xc9, 0x2c, 0x2e, 0x91, 0xd2, 0x4d, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x4f, 0xcf, 0x4f, 0xcf, 0xd7, 0x07, 0x2b, 0x49, 0x2a, 0x4d, 0x03, 0xf3, 0xc0, 0x1c, 0x30, 0x0b, 0xa2, 0x55, 0xc9, 0x9e, 0x8b, 0xd7, 0x3b, 0xb5, 0x32, 0x2c, 0x31, 0xa7, 0x34, 0x35, 0xb8, 0x24, 0xbf, 0x28, 0x55, 0x48, 0x8f, 0x8b, 0xb5, 0x20, 0x31, 0xb3, 0xa8, 0x58, 0x82, 0x51, 0x81, 0x59, 0x83, 0xdb, 0x48, - 0x42, 0x0f, 0x61, 0xb6, 0x1e, 0x4c, 0x65, 0x40, 0x62, 0x66, 0x51, 0x10, 0x44, 0x99, 0x92, 0x19, + 0x42, 0x0f, 0x61, 0xb6, 0x1e, 0x4c, 0x65, 0x40, 0x62, 0x66, 0x51, 0x10, 0x44, 0x99, 0x92, 0x0f, 0x17, 0x0f, 0xb2, 0xb0, 0x90, 0x00, 0x17, 0x73, 0x76, 0x6a, 0xa5, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x67, 0x10, 0x88, 0x29, 0x24, 0xc2, 0xc5, 0x5a, 0x06, 0x92, 0x96, 0x60, 0x52, 0x60, 0xd4, 0xe0, - 0x09, 0x82, 0x70, 0x9c, 0x4c, 0x2e, 0x3c, 0x94, 0x63, 0xb8, 0xf1, 0x50, 0x8e, 0xe1, 0xc3, 0x43, - 0x39, 0xc6, 0x86, 0x47, 0x72, 0x8c, 0x2b, 0x1e, 0xc9, 0x31, 0x9e, 0x78, 0x24, 0xc7, 0x78, 0xe1, - 0x91, 0x1c, 0xe3, 0x83, 0x47, 0x72, 0x8c, 0x2f, 0x1e, 0xc9, 0x31, 0x7c, 0x78, 0x24, 0xc7, 0x38, - 0xe1, 0xb1, 0x1c, 0xc3, 0x85, 0xc7, 0x72, 0x0c, 0x37, 0x1e, 0xcb, 0x31, 0x24, 0xb1, 0x81, 0x5d, - 0x6d, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x31, 0x79, 0x71, 0xc6, 0xfc, 0x00, 0x00, 0x00, + 0x09, 0x82, 0x70, 0x40, 0xa2, 0xc9, 0xf9, 0x29, 0xa9, 0xc9, 0x12, 0xcc, 0x60, 0x95, 0x10, 0x8e, + 0x93, 0xc9, 0x85, 0x87, 0x72, 0x0c, 0x37, 0x1e, 0xca, 0x31, 0x7c, 0x78, 0x28, 0xc7, 0xd8, 0xf0, + 0x48, 0x8e, 0x71, 0xc5, 0x23, 0x39, 0xc6, 0x13, 0x8f, 0xe4, 0x18, 0x2f, 0x3c, 0x92, 0x63, 0x7c, + 0xf0, 0x48, 0x8e, 0xf1, 0xc5, 0x23, 0x39, 0x86, 0x0f, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, 0x63, + 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, 0x39, 0x86, 0x24, 0x36, 0xb0, 0x5f, 0x8c, 0x01, 0x01, + 0x00, 0x00, 0xff, 0xff, 0x7a, 0x22, 0xdf, 0xec, 0x12, 0x01, 0x00, 0x00, } func (this *KeyValueStore) Equal(that interface{}) bool { @@ -200,6 +210,9 @@ func (this *KeyValuePair) Equal(that interface{}) bool { if !bytes.Equal(this.Value, that1.Value) { return false } + if this.Codec != that1.Codec { + return false + } return true } func (this *KeyValueStore) GoString() string { @@ -218,10 +231,11 @@ func (this *KeyValuePair) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) + s := make([]string, 0, 7) s = append(s, "&memberlist.KeyValuePair{") s = append(s, "Key: "+fmt.Sprintf("%#v", this.Key)+",\n") s = append(s, "Value: "+fmt.Sprintf("%#v", this.Value)+",\n") + s = append(s, "Codec: "+fmt.Sprintf("%#v", this.Codec)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -290,6 +304,12 @@ func (m *KeyValuePair) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintKv(dAtA, i, uint64(len(m.Value))) i += copy(dAtA[i:], m.Value) } + if len(m.Codec) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintKv(dAtA, i, uint64(len(m.Codec))) + i += copy(dAtA[i:], m.Codec) + } return i, nil } @@ -331,6 +351,10 @@ func (m *KeyValuePair) Size() (n int) { if l > 0 { n += 1 + l + sovKv(uint64(l)) } + l = len(m.Codec) + if l > 0 { + n += 1 + l + sovKv(uint64(l)) + } return n } @@ -364,6 +388,7 @@ func (this *KeyValuePair) String() string { s := strings.Join([]string{`&KeyValuePair{`, `Key:` + fmt.Sprintf("%v", this.Key) + `,`, `Value:` + fmt.Sprintf("%v", this.Value) + `,`, + `Codec:` + fmt.Sprintf("%v", this.Codec) + `,`, `}`, }, "") return s @@ -558,6 +583,38 @@ func (m *KeyValuePair) Unmarshal(dAtA []byte) error { m.Value = []byte{} } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Codec", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKv + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthKv + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Codec = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKv(dAtA[iNdEx:]) diff --git a/pkg/ring/kv/memberlist/kv.proto b/pkg/ring/kv/memberlist/kv.proto index 1d2b578a99d..cc5f12463b3 100644 --- a/pkg/ring/kv/memberlist/kv.proto +++ b/pkg/ring/kv/memberlist/kv.proto @@ -16,4 +16,7 @@ message KeyValueStore { message KeyValuePair { string key = 1; bytes value = 2; + + // ID of the codec used to write the value + string codec = 3; } diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index d75683e7d6e..b4307fddc10 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -26,30 +26,38 @@ const ( ) type Client struct { - kv *KV // reference to singleton memberlist-based KV + kv *KV // reference to singleton memberlist-based KV + codec codec.Codec } -func NewClient(kv *KV) *Client { +func NewClient(kv *KV, codec codec.Codec) *Client { + kv.RegisterCodec(codec) + return &Client{ - kv: kv, + kv: kv, + codec: codec, } } +// Get is part of kv.Client interface. func (c *Client) Get(ctx context.Context, key string) (interface{}, error) { - return c.kv.Get(ctx, key) + return c.kv.Get(key, c.codec) } +// CAS is part of kv.Client interface func (c *Client) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { - return c.kv.CAS(ctx, key, f) + return c.kv.CAS(ctx, key, c.codec, f) } +// WatchKey is part of kv.Client interface. func (c *Client) WatchKey(ctx context.Context, key string, f func(interface{}) bool) { - c.kv.WatchKey(ctx, key, f) + c.kv.WatchKey(ctx, key, c.codec, f) } // WatchPrefix calls f whenever any value stored under prefix changes. +// Part of kv.Client interface. func (c *Client) WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) { - c.kv.WatchPrefix(ctx, prefix, f) + c.kv.WatchPrefix(ctx, prefix, c.codec, f) } func (c *Client) Stop() { @@ -104,12 +112,15 @@ type KV struct { cfg KVConfig memberlist *memberlist.Memberlist broadcasts *memberlist.TransmitLimitedQueue - codec codec.Codec // KV Store. storeMu sync.Mutex store map[string]valueDesc + // Codec registry + codecsMu sync.RWMutex + codecs map[string]codec.Codec + // Key watchers watchersMu sync.Mutex watchers map[string][]chan string @@ -152,6 +163,9 @@ type valueDesc struct { // version (local only) is used to keep track of what we're gossiping about, and invalidate old messages version uint + + // ID of codec used to write this value. Only used when sending full state. + codecID string } var ( @@ -164,7 +178,7 @@ var ( // NewMemberlistClient creates new Client instance. If cfg.JoinMembers is set, it will also try to connect // to these members and join the cluster. If that fails and AbortIfJoinFails is true, error is returned and no // client is created. -func NewKV(cfg KVConfig, codec codec.Codec) (*KV, error) { +func NewKV(cfg KVConfig) (*KV, error) { level.Warn(util.Logger).Log("msg", "Using memberlist-based KV store is EXPERIMENTAL and not tested in production") cfg.TCPTransport.MetricsRegisterer = cfg.MetricsRegisterer @@ -207,9 +221,9 @@ func NewKV(cfg KVConfig, codec codec.Codec) (*KV, error) { memberlistClient := &KV{ cfg: cfg, store: make(map[string]valueDesc), + codecs: make(map[string]codec.Codec), watchers: make(map[string][]chan string), prefixWatchers: make(map[string][]chan string), - codec: codec, shutdown: make(chan struct{}), maxCasRetries: maxCasRetries, } @@ -249,6 +263,31 @@ func NewKV(cfg KVConfig, codec codec.Codec) (*KV, error) { return memberlistClient, nil } +// Registers codec to the map of codecs. +// +// We ignore duplicates -- we assume, that they all refer to the same codec. +// Unfortunately, we cannot verify that. There may be multiple instances of the same codec, +// so identity check doesn't work. +func (m *KV) RegisterCodec(codec codec.Codec) { + // should never happen in a tested code. + if codec.CodecID() == "" { + panic("invalid codec ID") + } + + m.codecsMu.Lock() + defer m.codecsMu.Unlock() + + m.codecs[codec.CodecID()] = codec +} + +// GetCodec returns codec for given ID or nil. +func (m *KV) GetCodec(codecID string) codec.Codec { + m.codecsMu.RLock() + defer m.codecsMu.RUnlock() + + return m.codecs[codecID] +} + // GetListeningPort returns port used for listening for memberlist communication. Useful when BindPort is set to 0. func (m *KV) GetListeningPort() int { return int(m.memberlist.LocalNode().Port) @@ -281,21 +320,21 @@ func (m *KV) Stop() { } // Get returns current value associated with given key. -// No communication with other nodes in the cluster is done here. Part of kv.Client interface. -func (m *KV) Get(ctx context.Context, key string) (interface{}, error) { - val, _, err := m.get(key) +// No communication with other nodes in the cluster is done here. +func (m *KV) Get(key string, codec codec.Codec) (interface{}, error) { + val, _, err := m.get(key, codec) return val, err } // Returns current value with removed tombstones. -func (m *KV) get(key string) (out interface{}, version uint, err error) { +func (m *KV) get(key string, codec codec.Codec) (out interface{}, version uint, err error) { m.storeMu.Lock() v := m.store[key] m.storeMu.Unlock() out = nil if v.value != nil { - out, err = m.codec.Decode(v.value) + out, err = codec.Decode(v.value) if err != nil { return nil, 0, err } @@ -314,9 +353,7 @@ func (m *KV) get(key string) (out interface{}, version uint, err error) { // latest value. Notifications that arrive while 'f' is running are coalesced into one subsequent 'f' call. // // Watching ends when 'f' returns false, context is done, or this client is shut down. -// -// Part of kv.Client interface. -func (m *KV) WatchKey(ctx context.Context, key string, f func(interface{}) bool) { +func (m *KV) WatchKey(ctx context.Context, key string, codec codec.Codec, f func(interface{}) bool) { // keep one extra notification, to avoid missing notification if we're busy running the function w := make(chan string, 1) @@ -337,7 +374,7 @@ func (m *KV) WatchKey(ctx context.Context, key string, f func(interface{}) bool) select { case <-w: // value changed - val, _, err := m.get(key) + val, _, err := m.get(key, codec) if err != nil { level.Warn(util.Logger).Log("msg", "failed to decode value while watching for changes", "key", key, "err", err) continue @@ -363,9 +400,7 @@ func (m *KV) WatchKey(ctx context.Context, key string, f func(interface{}) bool) // some notifications may be lost. // // Watching ends when 'f' returns false, context is done, or this client is shut down. -// -// Part of kv.Client interface. -func (m *KV) WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) { +func (m *KV) WatchPrefix(ctx context.Context, prefix string, codec codec.Codec, f func(string, interface{}) bool) { // we use bigger buffer here, since keys are interesting and we don't want to lose them. w := make(chan string, 16) @@ -385,7 +420,7 @@ func (m *KV) WatchPrefix(ctx context.Context, prefix string, f func(string, inte for { select { case key := <-w: - val, _, err := m.get(key) + val, _, err := m.get(key, codec) if err != nil { level.Warn(util.Logger).Log("msg", "failed to decode value while watching for changes", "key", key, "err", err) continue @@ -455,7 +490,7 @@ func (m *KV) notifyWatchers(key string) { } } -// CAS is part of kv.Client implementation. +// CAS implements Compare-And-Set/Swap operation. // // CAS expects that value returned by 'f' function implements Mergeable interface. If it doesn't, CAS fails immediately. // @@ -464,7 +499,7 @@ func (m *KV) notifyWatchers(key string) { // KV store, and change is broadcast to cluster peers. Merge function is called with CAS flag on, so that it can // detect removals. If Merge doesn't result in any change (returns nil), then operation fails and is retried again. // After too many failed retries, this method returns error. -func (m *KV) CAS(ctx context.Context, key string, f func(in interface{}) (out interface{}, retry bool, err error)) error { +func (m *KV) CAS(ctx context.Context, key string, codec codec.Codec, f func(in interface{}) (out interface{}, retry bool, err error)) error { var lastError error = nil outer: @@ -485,7 +520,7 @@ outer: } } - change, newver, retry, err := m.trySingleCas(key, f) + change, newver, retry, err := m.trySingleCas(key, codec, f) if err != nil { level.Debug(util.Logger).Log("msg", "CAS attempt failed", "err", err, "retry", retry) @@ -499,7 +534,7 @@ outer: if change != nil { m.casSuccesses.Inc() m.notifyWatchers(key) - m.broadcastNewValue(key, change, newver) + m.broadcastNewValue(key, change, newver, codec) } return nil @@ -516,8 +551,8 @@ outer: // returns change, error (or nil, if CAS succeeded), and whether to retry or not. // returns errNoChangeDetected if merge failed to detect change in f's output. -func (m *KV) trySingleCas(key string, f func(in interface{}) (out interface{}, retry bool, err error)) (Mergeable, uint, bool, error) { - val, ver, err := m.get(key) +func (m *KV) trySingleCas(key string, codec codec.Codec, f func(in interface{}) (out interface{}, retry bool, err error)) (Mergeable, uint, bool, error) { + val, ver, err := m.get(key, codec) if err != nil { return nil, 0, false, fmt.Errorf("failed to get value: %v", err) } @@ -540,7 +575,7 @@ func (m *KV) trySingleCas(key string, f func(in interface{}) (out interface{}, r // To support detection of removed items from value, we will only allow CAS operation to // succeed if version hasn't changed, i.e. state hasn't changed since running 'f'. - change, newver, err := m.mergeValueForKey(key, r, ver) + change, newver, err := m.mergeValueForKey(key, r, ver, codec) if err == errVersionMismatch { return nil, 0, retry, err } @@ -557,14 +592,14 @@ func (m *KV) trySingleCas(key string, f func(in interface{}) (out interface{}, r return change, newver, retry, nil } -func (m *KV) broadcastNewValue(key string, change Mergeable, version uint) { - data, err := m.codec.Encode(change) +func (m *KV) broadcastNewValue(key string, change Mergeable, version uint, codec codec.Codec) { + data, err := codec.Encode(change) if err != nil { level.Error(util.Logger).Log("msg", "failed to encode change", "err", err) return } - kvPair := KeyValuePair{Key: key, Value: data} + kvPair := KeyValuePair{Key: key, Value: data, Codec: codec.CodecID()} pairData, err := kvPair.Marshal() if err != nil { level.Error(util.Logger).Log("msg", "failed to serialize KV pair", "err", err) @@ -611,8 +646,14 @@ func (m *KV) NotifyMsg(msg []byte) { return } + codec := m.GetCodec(kvPair.GetCodec()) + if codec == nil { + level.Error(util.Logger).Log("msg", "failed to decode received value, unknown codec", "codec", kvPair.GetCodec()) + return + } + // we have a ring update! Let's merge it with our version of the ring for given key - mod, version, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value) + mod, version, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value, codec) if err != nil { level.Error(util.Logger).Log("msg", "failed to store received value", "key", kvPair.Key, "err", err) } else if version > 0 { @@ -675,6 +716,7 @@ func (m *KV) LocalState(join bool) []byte { kvPair.Reset() kvPair.Key = key kvPair.Value = val.value + kvPair.Codec = val.codecID ser, err := kvPair.Marshal() if err != nil { @@ -733,13 +775,19 @@ func (m *KV) MergeRemoteState(data []byte, join bool) { data = data[kvPairLength:] + codec := m.GetCodec(kvPair.GetCodec()) + if codec == nil { + err = fmt.Errorf("unknown codec: %s", kvPair.GetCodec()) + break + } + // we have both key and value, try to merge it with our state - change, newver, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value) + change, newver, err := m.mergeBytesValueForKey(kvPair.Key, kvPair.Value, codec) if err != nil { level.Error(util.Logger).Log("msg", "failed to store received value", "key", kvPair.Key, "err", err) } else if newver > 0 { m.notifyWatchers(kvPair.Key) - m.broadcastNewValue(kvPair.Key, change, newver) + m.broadcastNewValue(kvPair.Key, change, newver, codec) } } @@ -748,8 +796,8 @@ func (m *KV) MergeRemoteState(data []byte, join bool) { } } -func (m *KV) mergeBytesValueForKey(key string, incomingData []byte) (Mergeable, uint, error) { - decodedValue, err := m.codec.Decode(incomingData) +func (m *KV) mergeBytesValueForKey(key string, incomingData []byte, codec codec.Codec) (Mergeable, uint, error) { + decodedValue, err := codec.Decode(incomingData) if err != nil { return nil, 0, fmt.Errorf("failed to decode value: %v", err) } @@ -759,14 +807,14 @@ func (m *KV) mergeBytesValueForKey(key string, incomingData []byte) (Mergeable, return nil, 0, fmt.Errorf("expected Mergeable, got: %T", decodedValue) } - return m.mergeValueForKey(key, incomingValue, 0) + return m.mergeValueForKey(key, incomingValue, 0, codec) } // Merges incoming value with value we have in our store. Returns "a change" that can be sent to other // cluster members to update their state, and new version of the value. // If CAS version is specified, then merging will fail if state has changed already, and errVersionMismatch is reported. // If no modification occurred, new version is 0. -func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion uint) (Mergeable, uint, error) { +func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion uint, codec codec.Codec) (Mergeable, uint, error) { m.storeMu.Lock() defer m.storeMu.Unlock() @@ -775,7 +823,7 @@ func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion ui if casVersion > 0 && curr.version != casVersion { return nil, 0, errVersionMismatch } - result, change, err := computeNewValue(incomingValue, curr.value, m.codec, casVersion > 0) + result, change, err := computeNewValue(incomingValue, curr.value, codec, casVersion > 0) if err != nil { return nil, 0, err } @@ -790,7 +838,7 @@ func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion ui result.RemoveTombstones(limit) } - encoded, err := m.codec.Encode(result) + encoded, err := codec.Encode(result) if err != nil { return nil, 0, fmt.Errorf("failed to encode merged result: %v", err) } @@ -799,6 +847,7 @@ func (m *KV) mergeValueForKey(key string, incomingValue Mergeable, casVersion ui m.store[key] = valueDesc{ value: encoded, version: newVersion, + codecID: codec.CodecID(), } return change, newVersion, nil diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index f557d74a974..504b2e99778 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -8,14 +8,11 @@ import ( "fmt" "math" "math/rand" - "os" "sort" "sync" "testing" "time" - "github.com/go-kit/kit/log" - "github.com/go-kit/kit/log/level" "github.com/stretchr/testify/require" "github.com/cortexproject/cortex/pkg/ring/kv/codec" @@ -210,12 +207,12 @@ func TestBasicGetAndCas(t *testing.T) { }, } - mkv, err := NewKV(cfg, c) + mkv, err := NewKV(cfg) if err != nil { t.Fatal("Failed to setup KV client", err) } defer mkv.Stop() - kv := NewClient(mkv) + kv := NewClient(mkv, c) const key = "test" @@ -265,12 +262,12 @@ func withFixtures(t *testing.T, testFN func(t *testing.T, kv *Client)) { TCPTransport: TCPTransportConfig{}, } - mkv, err := NewKV(cfg, c) + mkv, err := NewKV(cfg) if err != nil { t.Fatal("Failed to setup KV client", err) } defer mkv.Stop() - kv := NewClient(mkv) + kv := NewClient(mkv, c) testFN(t, kv) } @@ -407,12 +404,12 @@ func TestMultipleCAS(t *testing.T) { cfg := KVConfig{} - mkv, err := NewKV(cfg, c) + mkv, err := NewKV(cfg) if err != nil { t.Fatal("Failed to setup KV client", err) } mkv.maxCasRetries = 20 - kv := NewClient(mkv) + kv := NewClient(mkv, c) defer kv.Stop() wg := &sync.WaitGroup{} @@ -481,8 +478,6 @@ func TestMultipleCAS(t *testing.T) { func TestMultipleClients(t *testing.T) { c := dataCodec{} - l := log.NewLogfmtLogger(os.Stdout) - l = level.NewFilter(l, level.AllowInfo()) const members = 10 const key = "ring" @@ -514,11 +509,11 @@ func TestMultipleClients(t *testing.T) { }, } - mkv, err := NewKV(cfg, c) + mkv, err := NewKV(cfg) if err != nil { t.Fatal(id, "Failed to setup KV client", err) } - kv := NewClient(mkv) + kv := NewClient(mkv, c) clients = append(clients, kv) From 80a0c157a3ae2244c8d0b0abef73c70a687e47cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 13:17:45 +0100 Subject: [PATCH 05/29] Added tests for multi-codecs KV. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- .../kv/memberlist/memberlist_client_test.go | 137 ++++++++++++++++++ 1 file changed, 137 insertions(+) diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index 504b2e99778..456d9cee97f 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -660,3 +660,140 @@ func generateTokens(numTokens int) []uint32 { } return tokens } + +type distributedCounter map[string]int + +func (dc distributedCounter) Merge(mergeable Mergeable, localCAS bool) (Mergeable, error) { + if mergeable == nil { + return nil, nil + } + + odc, ok := mergeable.(distributedCounter) + if !ok || odc == nil { + return nil, fmt.Errorf("invalid thing to merge: %T", mergeable) + } + + updated := distributedCounter{} + + for k, v := range odc { + if v > dc[k] { + dc[k] = v + updated[k] = v + } + } + + if len(updated) == 0 { + return nil, nil + } + return updated, nil +} + +func (dc distributedCounter) MergeContent() []string { + // return list of keys + out := []string(nil) + for k := range dc { + out = append(out, k) + } + return out +} + +func (dc distributedCounter) RemoveTombstones(limit time.Time) { + // nothing to do +} + +type distributedCounterCodec struct{} + +func (d distributedCounterCodec) CodecID() string { + return "distributedCounter" +} + +func (d distributedCounterCodec) Decode(b []byte) (interface{}, error) { + dec := gob.NewDecoder(bytes.NewBuffer(b)) + out := &distributedCounter{} + err := dec.Decode(out) + return *out, err +} + +func (d distributedCounterCodec) Encode(val interface{}) ([]byte, error) { + buf := bytes.Buffer{} + enc := gob.NewEncoder(&buf) + err := enc.Encode(val) + return buf.Bytes(), err +} + +var _ codec.Codec = &distributedCounterCodec{} + +func TestMultipleCodecs(t *testing.T) { + cfg := KVConfig{ + TCPTransport: TCPTransportConfig{ + BindAddrs: []string{"localhost"}, + BindPort: 0, // randomize + }, + } + + mkv1, err := NewKV(cfg) + require.NoError(t, err) + defer mkv1.Stop() + + kv1 := NewClient(mkv1, dataCodec{}) + kv2 := NewClient(mkv1, distributedCounterCodec{}) + + err = kv1.CAS(context.Background(), "data", func(in interface{}) (out interface{}, retry bool, err error) { + var d *data = nil + if in != nil { + d = in.(*data) + } + if d == nil { + d = &data{} + } + if d.Members == nil { + d.Members = map[string]member{} + } + d.Members["test"] = member{ + Timestamp: time.Now().Unix(), + State: ACTIVE, + } + return d, true, nil + }) + require.NoError(t, err) + + err = kv2.CAS(context.Background(), "counter", func(in interface{}) (out interface{}, retry bool, err error) { + var dc distributedCounter = nil + if in != nil { + dc = in.(distributedCounter) + } + if dc == nil { + dc = distributedCounter{} + } + dc["test"] = 5 + return dc, true, err + }) + require.NoError(t, err) + + // We will read values from second KV, which will join the first one + mkv2, err := NewKV(cfg) + require.NoError(t, err) + defer mkv2.Stop() + + // We need to register codec to second KV. Normally client does that, but we don't have any client for second KV. + mkv2.RegisterCodec(dataCodec{}) + mkv2.RegisterCodec(distributedCounterCodec{}) + + // Join second KV to first one. That will also trigger state transfer. + _, err = mkv2.JoinMembers([]string{fmt.Sprintf("127.0.0.1:%d", mkv1.GetListeningPort())}) + require.NoError(t, err) + + // Now read both values from second KV. It should have both values by now. + + // fetch directly from single KV, to see that both are stored in the same one + val, err := mkv2.Get("data", dataCodec{}) + require.NoError(t, err) + require.NotNil(t, val) + require.NotZero(t, val.(*data).Members["test"].Timestamp) + require.Equal(t, ACTIVE, val.(*data).Members["test"].State) + + val, err = mkv2.Get("counter", distributedCounterCodec{}) + require.NoError(t, err) + require.NotNil(t, val) + require.Equal(t, 5, val.(distributedCounter)["test"]) +} From fd6f816b390e1415bdeea6d437cb03b800f8aaa7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 13:56:25 +0100 Subject: [PATCH 06/29] Fixed client creation. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 54737d8ffe1..6d7995680ff 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -125,14 +125,14 @@ func createClient(name string, prefix string, cfg StoreConfig, codec codec.Codec case "memberlist": cfg.Memberlist.MetricsRegisterer = prometheus.DefaultRegisterer singletonKVStoreInit.Do(func() { - singletonKVStore, singletonKVStoreError = memberlist.NewKV(cfg.Memberlist, codec) + singletonKVStore, singletonKVStoreError = memberlist.NewKV(cfg.Memberlist) }) kv, err := singletonKVStore, singletonKVStoreError if err != nil { return nil, err } - client = memberlist.NewClient(kv) + client = memberlist.NewClient(kv, codec) case "multi": client, err = buildMultiClient(cfg, codec) From 13a3b3da0aee051dfbba52a66e8571dcb764ebcb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 14:04:40 +0100 Subject: [PATCH 07/29] Removed Stop method from kv.Client interface. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It was introduced for memberlist Client, but it will not be needed anymore as memberlist.KV will be a top-level component with its own lifecycle, separated from the client. Signed-off-by: Peter Štibraný --- pkg/ring/kv/client.go | 3 --- pkg/ring/kv/consul/client.go | 5 ----- pkg/ring/kv/etcd/etcd.go | 5 ----- pkg/ring/kv/memberlist/memberlist_client.go | 4 ---- pkg/ring/kv/memberlist/memberlist_client_test.go | 5 +++-- pkg/ring/kv/metrics.go | 4 ---- pkg/ring/kv/multi.go | 9 --------- pkg/ring/kv/prefix.go | 4 ---- pkg/ring/lifecycler.go | 2 -- pkg/ring/lifecycler_test.go | 4 ---- pkg/ring/ring.go | 2 -- 11 files changed, 3 insertions(+), 44 deletions(-) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 6d7995680ff..76598bdd662 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -88,9 +88,6 @@ type Client interface { // WatchPrefix calls f whenever any value stored under prefix changes. WatchPrefix(ctx context.Context, prefix string, f func(string, interface{}) bool) - - // If client needs to do some cleanup, it can do it here. - Stop() } // NewClient creates a new Client (consul, etcd or inmemory) based on the config, diff --git a/pkg/ring/kv/consul/client.go b/pkg/ring/kv/consul/client.go index a4ea87a0b9b..d8cfb1ffe21 100644 --- a/pkg/ring/kv/consul/client.go +++ b/pkg/ring/kv/consul/client.go @@ -328,8 +328,3 @@ func (c *Client) createRateLimiter() *rate.Limiter { } return rate.NewLimiter(rate.Limit(c.cfg.WatchKeyRateLimit), burst) } - -// Stop does nothing in Consul client. -func (c *Client) Stop() { - // nothing to do -} diff --git a/pkg/ring/kv/etcd/etcd.go b/pkg/ring/kv/etcd/etcd.go index da0494533cb..95599ad85b9 100644 --- a/pkg/ring/kv/etcd/etcd.go +++ b/pkg/ring/kv/etcd/etcd.go @@ -194,8 +194,3 @@ func (c *Client) Get(ctx context.Context, key string) (interface{}, error) { } return c.codec.Decode(resp.Kvs[0].Value) } - -// Stop does nothing in etcd client. -func (c *Client) Stop() { - // nothing to do -} diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index b4307fddc10..6fb9a54858d 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -60,10 +60,6 @@ func (c *Client) WatchPrefix(ctx context.Context, prefix string, f func(string, c.kv.WatchPrefix(ctx, prefix, c.codec, f) } -func (c *Client) Stop() { - c.kv.Stop() -} - // Config for memberlist-based Client type KVConfig struct { // Memberlist options. diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index 456d9cee97f..56bdd5ab080 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -409,8 +409,9 @@ func TestMultipleCAS(t *testing.T) { t.Fatal("Failed to setup KV client", err) } mkv.maxCasRetries = 20 + defer mkv.Stop() + kv := NewClient(mkv, c) - defer kv.Stop() wg := &sync.WaitGroup{} start := make(chan struct{}) @@ -627,7 +628,7 @@ func getTimestamps(members map[string]member) (min int64, max int64, avg int64) func runClient(t *testing.T, kv *Client, name string, ringKey string, portToConnect int, start <-chan struct{}, stop <-chan struct{}) { // stop gossipping about the ring(s) - defer kv.Stop() + defer kv.kv.Stop() for { select { diff --git a/pkg/ring/kv/metrics.go b/pkg/ring/kv/metrics.go index 913af80743a..37fb4fc9e04 100644 --- a/pkg/ring/kv/metrics.go +++ b/pkg/ring/kv/metrics.go @@ -64,7 +64,3 @@ func (m metrics) WatchPrefix(ctx context.Context, prefix string, f func(string, return nil }) } - -func (m metrics) Stop() { - m.c.Stop() -} diff --git a/pkg/ring/kv/multi.go b/pkg/ring/kv/multi.go index 3e21f7cfe89..11b4d68b2e5 100644 --- a/pkg/ring/kv/multi.go +++ b/pkg/ring/kv/multi.go @@ -351,12 +351,3 @@ func (m *MultiClient) writeToSecondary(ctx context.Context, primary kvclient, ke } } } - -// Stop the multiClient and all configured clients. -func (m *MultiClient) Stop() { - m.cancel() - - for _, kv := range m.clients { - kv.client.Stop() - } -} diff --git a/pkg/ring/kv/prefix.go b/pkg/ring/kv/prefix.go index be9a185c854..d5c21d2c856 100644 --- a/pkg/ring/kv/prefix.go +++ b/pkg/ring/kv/prefix.go @@ -37,7 +37,3 @@ func (c *prefixedKVClient) WatchPrefix(ctx context.Context, prefix string, f fun func (c *prefixedKVClient) Get(ctx context.Context, key string) (interface{}, error) { return c.client.Get(ctx, c.prefix+key) } - -func (c *prefixedKVClient) Stop() { - c.client.Stop() -} diff --git a/pkg/ring/lifecycler.go b/pkg/ring/lifecycler.go index bd1841a4368..45b67b7d475 100644 --- a/pkg/ring/lifecycler.go +++ b/pkg/ring/lifecycler.go @@ -453,8 +453,6 @@ heartbeatLoop: } level.Info(util.Logger).Log("msg", "instance removed from the KV store", "ring", i.RingName) } - - i.KVStore.Stop() } // initRing is the first thing we do when we start. It: diff --git a/pkg/ring/lifecycler_test.go b/pkg/ring/lifecycler_test.go index cf41e1c842b..3bfcae4520b 100644 --- a/pkg/ring/lifecycler_test.go +++ b/pkg/ring/lifecycler_test.go @@ -233,10 +233,6 @@ func (m *MockClient) WatchPrefix(ctx context.Context, prefix string, f func(stri } } -func (m *MockClient) Stop() { - // nothing to do -} - // Ensure a check ready returns error when consul returns a nil key and the ingester already holds keys. This happens if the ring key gets deleted func TestCheckReady(t *testing.T) { var ringConfig Config diff --git a/pkg/ring/ring.go b/pkg/ring/ring.go index 19f3f0283a9..4b98a481912 100644 --- a/pkg/ring/ring.go +++ b/pkg/ring/ring.go @@ -181,8 +181,6 @@ func (r *Ring) loop(ctx context.Context) { r.ringTokens = ringTokens return true }) - - r.KVClient.Stop() } // Get returns n (or more) ingesters which form the replicas for the given key. From bd9f7975468785cf753dd8c2dc68aafa9ae629d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 14:27:53 +0100 Subject: [PATCH 08/29] Memberlist KV is now top-level component. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It is initialized if any component actually uses it, but not sooner. If initialized, Stop method is called when server shuts down. Signed-off-by: Peter Štibraný --- pkg/cortex/cortex.go | 14 +++++++++----- pkg/cortex/memberlist_kv.go | 28 ++++++++++++++++++++++++++++ pkg/cortex/modules.go | 28 ++++++++++++++++++++++++++-- pkg/ring/kv/client.go | 23 ++++++----------------- 4 files changed, 69 insertions(+), 24 deletions(-) create mode 100644 pkg/cortex/memberlist_kv.go diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 2880362c851..da5da232414 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -30,6 +30,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/frontend" "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/cortexproject/cortex/pkg/ring" + "github.com/cortexproject/cortex/pkg/ring/kv/memberlist" "github.com/cortexproject/cortex/pkg/ruler" "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/util" @@ -84,6 +85,7 @@ type Config struct { ConfigStore config_client.Config `yaml:"config_store,omitempty"` Alertmanager alertmanager.MultitenantAlertmanagerConfig `yaml:"alertmanager,omitempty"` RuntimeConfig runtimeconfig.ManagerConfig `yaml:"runtime_config,omitempty"` + MemberlistKV memberlist.KVConfig `yaml:"memberlist_kv"` } // RegisterFlags registers flag. @@ -119,6 +121,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.ConfigStore.RegisterFlagsWithPrefix("alertmanager.", f) c.Alertmanager.RegisterFlags(f) c.RuntimeConfig.RegisterFlags(f) + c.MemberlistKV.RegisterFlags(f, "") // These don't seem to have a home. flag.IntVar(&chunk_util.QueryParallelism, "querier.query-parallelism", 100, "Max subqueries run in parallel per higher-level query.") @@ -171,11 +174,12 @@ type Cortex struct { cache cache.Cache runtimeConfig *runtimeconfig.Manager - ruler *ruler.Ruler - configAPI *api.API - configDB db.DB - alertmanager *alertmanager.MultitenantAlertmanager - compactor *compactor.Compactor + ruler *ruler.Ruler + configAPI *api.API + configDB db.DB + alertmanager *alertmanager.MultitenantAlertmanager + compactor *compactor.Compactor + memberlistKVState *memberlistKVState // The chunk store that the querier should use to query the long // term storage. It depends on the storage engine used. diff --git a/pkg/cortex/memberlist_kv.go b/pkg/cortex/memberlist_kv.go new file mode 100644 index 00000000000..773efca42b0 --- /dev/null +++ b/pkg/cortex/memberlist_kv.go @@ -0,0 +1,28 @@ +package cortex + +import ( + "sync" + + "github.com/cortexproject/cortex/pkg/ring/kv/memberlist" +) + +type memberlistKVState struct { + cfg *memberlist.KVConfig + init sync.Once + kv *memberlist.KV + err error +} + +func newMemberlistKVState(cfg *memberlist.KVConfig) *memberlistKVState { + return &memberlistKVState{ + cfg: cfg, + } +} + +func (kvs *memberlistKVState) getMemberlistKV() (*memberlist.KV, error) { + kvs.init.Do(func() { + kvs.kv, kvs.err = memberlist.NewKV(*kvs.cfg) + }) + + return kvs.kv, kvs.err +} diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 07dab65361f..0aeb3490889 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -56,6 +56,7 @@ const ( Configs AlertManager Compactor + MemberlistKV All ) @@ -91,6 +92,8 @@ func (m moduleName) String() string { return "alertmanager" case Compactor: return "compactor" + case MemberlistKV: + return "memberlist-kv" case All: return "all" default: @@ -174,6 +177,7 @@ func (t *Cortex) stopServer() (err error) { func (t *Cortex) initRing(cfg *Config) (err error) { cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) + cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.memberlistKVState.getMemberlistKV t.ring, err = ring.New(cfg.Ingester.LifecyclerConfig.RingConfig, "ingester", ring.IngesterRingKey) if err != nil { return @@ -306,6 +310,7 @@ func (t *Cortex) stopQuerierChunkStore() error { func (t *Cortex) initIngester(cfg *Config) (err error) { cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) + cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.memberlistKVState.getMemberlistKV cfg.Ingester.LifecyclerConfig.ListenPort = &cfg.Server.GRPCListenPort cfg.Ingester.TSDBEnabled = cfg.Storage.Engine == storage.StorageEngineTSDB cfg.Ingester.TSDBConfig = cfg.TSDB @@ -510,6 +515,20 @@ func (t *Cortex) stopCompactor() error { return nil } +func (t *Cortex) initMemberlistKV(cfg *Config) (err error) { + cfg.MemberlistKV.MetricsRegisterer = prometheus.DefaultRegisterer + t.memberlistKVState = newMemberlistKVState(&cfg.MemberlistKV) + return nil +} + +func (t *Cortex) stopMemberlistKV() (err error) { + kv := t.memberlistKVState.kv + if kv != nil { + kv.Stop() + } + return nil +} + type module struct { deps []moduleName init func(t *Cortex, cfg *Config) error @@ -527,8 +546,13 @@ var modules = map[moduleName]module{ stop: (*Cortex).stopRuntimeConfig, }, + MemberlistKV: { + init: (*Cortex).initMemberlistKV, + stop: (*Cortex).stopMemberlistKV, + }, + Ring: { - deps: []moduleName{Server, RuntimeConfig}, + deps: []moduleName{Server, RuntimeConfig, MemberlistKV}, init: (*Cortex).initRing, }, @@ -550,7 +574,7 @@ var modules = map[moduleName]module{ }, Ingester: { - deps: []moduleName{Overrides, Store, Server, RuntimeConfig}, + deps: []moduleName{Overrides, Store, Server, RuntimeConfig, MemberlistKV}, init: (*Cortex).initIngester, stop: (*Cortex).stopIngester, }, diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 76598bdd662..20d2c013b5f 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -6,8 +6,6 @@ import ( "fmt" "sync" - "github.com/prometheus/client_golang/prometheus" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/ring/kv/etcd" @@ -20,18 +18,15 @@ import ( var inmemoryStoreInit sync.Once var inmemoryStore Client -var singletonKVStoreInit sync.Once -var singletonKVStore *memberlist.KV -var singletonKVStoreError error - // StoreConfig is a configuration used for building single store client, either // Consul, Etcd, Memberlist or MultiClient. It was extracted from Config to keep // single-client config separate from final client-config (with all the wrappers) type StoreConfig struct { - Consul consul.Config `yaml:"consul,omitempty"` - Etcd etcd.Config `yaml:"etcd,omitempty"` - Memberlist memberlist.KVConfig `yaml:"memberlist,omitempty"` - Multi MultiConfig `yaml:"multi,omitempty"` + Consul consul.Config `yaml:"consul,omitempty"` + Etcd etcd.Config `yaml:"etcd,omitempty"` + Multi MultiConfig `yaml:"multi,omitempty"` + + MemberlistKV func() (*memberlist.KV, error) } // Config is config for a KVStore currently used by ring and HA tracker, @@ -57,7 +52,6 @@ func (cfg *Config) RegisterFlagsWithPrefix(flagsPrefix, defaultPrefix string, f cfg.Consul.RegisterFlags(f, flagsPrefix) cfg.Etcd.RegisterFlagsWithPrefix(f, flagsPrefix) cfg.Multi.RegisterFlagsWithPrefix(f, flagsPrefix) - cfg.Memberlist.RegisterFlags(f, flagsPrefix) if flagsPrefix == "" { flagsPrefix = "ring." @@ -120,12 +114,7 @@ func createClient(name string, prefix string, cfg StoreConfig, codec codec.Codec client = inmemoryStore case "memberlist": - cfg.Memberlist.MetricsRegisterer = prometheus.DefaultRegisterer - singletonKVStoreInit.Do(func() { - singletonKVStore, singletonKVStoreError = memberlist.NewKV(cfg.Memberlist) - }) - - kv, err := singletonKVStore, singletonKVStoreError + kv, err := cfg.MemberlistKV() if err != nil { return nil, err } From 1893c53e1448f20714e47b2de35044469decb8e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 20:38:42 +0100 Subject: [PATCH 09/29] All codecs are registered to memberlist KV at the beginning. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Client now only verifies that KV knows about coded that client wants to use, but doesn't register it anymore. Signed-off-by: Peter Štibraný --- pkg/cortex/modules.go | 4 ++ pkg/ring/kv/client.go | 2 +- pkg/ring/kv/memberlist/memberlist_client.go | 43 ++++++--------- .../kv/memberlist/memberlist_client_test.go | 55 +++++++++++-------- 4 files changed, 53 insertions(+), 51 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 0aeb3490889..7fd8717494b 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -31,6 +31,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/frontend" "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/cortexproject/cortex/pkg/ring" + "github.com/cortexproject/cortex/pkg/ring/kv/codec" "github.com/cortexproject/cortex/pkg/ruler" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/runtimeconfig" @@ -517,6 +518,9 @@ func (t *Cortex) stopCompactor() error { func (t *Cortex) initMemberlistKV(cfg *Config) (err error) { cfg.MemberlistKV.MetricsRegisterer = prometheus.DefaultRegisterer + cfg.MemberlistKV.Codecs = []codec.Codec{ + ring.GetCodec(), + } t.memberlistKVState = newMemberlistKVState(&cfg.MemberlistKV) return nil } diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 20d2c013b5f..7e8de46732c 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -118,7 +118,7 @@ func createClient(name string, prefix string, cfg StoreConfig, codec codec.Codec if err != nil { return nil, err } - client = memberlist.NewClient(kv, codec) + client, err = memberlist.NewClient(kv, codec) case "multi": client, err = buildMultiClient(cfg, codec) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 6fb9a54858d..fc77f2f1065 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -30,13 +30,16 @@ type Client struct { codec codec.Codec } -func NewClient(kv *KV, codec codec.Codec) *Client { - kv.RegisterCodec(codec) +func NewClient(kv *KV, codec codec.Codec) (*Client, error) { + c := kv.GetCodec(codec.CodecID()) + if c == nil { + return nil, fmt.Errorf("codec not registered in KV: %s", codec.CodecID()) + } return &Client{ kv: kv, codec: codec, - } + }, nil } // Get is part of kv.Client interface. @@ -85,6 +88,9 @@ type KVConfig struct { // Where to put custom metrics. Metrics are not registered, if this is nil. MetricsRegisterer prometheus.Registerer `yaml:"-"` MetricsNamespace string `yaml:"-"` + + // Codecs to register. Codecs need to be registered before joining other members. + Codecs []codec.Codec } // RegisterFlags registers flags. @@ -114,8 +120,7 @@ type KV struct { store map[string]valueDesc // Codec registry - codecsMu sync.RWMutex - codecs map[string]codec.Codec + codecs map[string]codec.Codec // Key watchers watchersMu sync.Mutex @@ -241,6 +246,10 @@ func NewKV(cfg KVConfig) (*KV, error) { // Almost ready... memberlistClient.createAndRegisterMetrics() + for _, c := range cfg.Codecs { + memberlistClient.codecs[c.CodecID()] = c + } + // Join the cluster if len(cfg.JoinMembers) > 0 { reached, err := memberlistClient.JoinMembers(cfg.JoinMembers) @@ -259,28 +268,8 @@ func NewKV(cfg KVConfig) (*KV, error) { return memberlistClient, nil } -// Registers codec to the map of codecs. -// -// We ignore duplicates -- we assume, that they all refer to the same codec. -// Unfortunately, we cannot verify that. There may be multiple instances of the same codec, -// so identity check doesn't work. -func (m *KV) RegisterCodec(codec codec.Codec) { - // should never happen in a tested code. - if codec.CodecID() == "" { - panic("invalid codec ID") - } - - m.codecsMu.Lock() - defer m.codecsMu.Unlock() - - m.codecs[codec.CodecID()] = codec -} - // GetCodec returns codec for given ID or nil. func (m *KV) GetCodec(codecID string) codec.Codec { - m.codecsMu.RLock() - defer m.codecsMu.RUnlock() - return m.codecs[codecID] } @@ -773,8 +762,8 @@ func (m *KV) MergeRemoteState(data []byte, join bool) { codec := m.GetCodec(kvPair.GetCodec()) if codec == nil { - err = fmt.Errorf("unknown codec: %s", kvPair.GetCodec()) - break + level.Error(util.Logger).Log("msg", "failed to parse remote state: unknown codec for key", "codec", kvPair.GetCodec(), "key", kvPair.GetKey()) + continue } // we have both key and value, try to merge it with our state diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index 56bdd5ab080..696cbf2ea07 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -205,14 +205,15 @@ func TestBasicGetAndCas(t *testing.T) { TCPTransport: TCPTransportConfig{ BindAddrs: []string{"localhost"}, }, + Codecs: []codec.Codec{c}, } mkv, err := NewKV(cfg) - if err != nil { - t.Fatal("Failed to setup KV client", err) - } + require.NoError(t, err) defer mkv.Stop() - kv := NewClient(mkv, c) + + kv, err := NewClient(mkv, c) + require.NoError(t, err) const key = "test" @@ -260,14 +261,15 @@ func withFixtures(t *testing.T, testFN func(t *testing.T, kv *Client)) { cfg := KVConfig{ TCPTransport: TCPTransportConfig{}, + Codecs: []codec.Codec{c}, } mkv, err := NewKV(cfg) - if err != nil { - t.Fatal("Failed to setup KV client", err) - } + require.NoError(t, err) defer mkv.Stop() - kv := NewClient(mkv, c) + + kv, err := NewClient(mkv, c) + require.NoError(t, err) testFN(t, kv) } @@ -402,16 +404,17 @@ func TestCASFailedBecauseOfVersionChanges(t *testing.T) { func TestMultipleCAS(t *testing.T) { c := dataCodec{} - cfg := KVConfig{} + cfg := KVConfig{ + Codecs: []codec.Codec{c}, + } mkv, err := NewKV(cfg) - if err != nil { - t.Fatal("Failed to setup KV client", err) - } + require.NoError(t, err) mkv.maxCasRetries = 20 defer mkv.Stop() - kv := NewClient(mkv, c) + kv, err := NewClient(mkv, c) + require.NoError(t, err) wg := &sync.WaitGroup{} start := make(chan struct{}) @@ -508,13 +511,15 @@ func TestMultipleClients(t *testing.T) { BindAddrs: []string{"localhost"}, BindPort: 0, // randomize ports }, + + Codecs: []codec.Codec{c}, } mkv, err := NewKV(cfg) - if err != nil { - t.Fatal(id, "Failed to setup KV client", err) - } - kv := NewClient(mkv, c) + require.NoError(t, err) + + kv, err := NewClient(mkv, c) + require.NoError(t, err) clients = append(clients, kv) @@ -730,14 +735,22 @@ func TestMultipleCodecs(t *testing.T) { BindAddrs: []string{"localhost"}, BindPort: 0, // randomize }, + + Codecs: []codec.Codec{ + dataCodec{}, + distributedCounterCodec{}, + }, } mkv1, err := NewKV(cfg) require.NoError(t, err) defer mkv1.Stop() - kv1 := NewClient(mkv1, dataCodec{}) - kv2 := NewClient(mkv1, distributedCounterCodec{}) + kv1, err := NewClient(mkv1, dataCodec{}) + require.NoError(t, err) + + kv2, err := NewClient(mkv1, distributedCounterCodec{}) + require.NoError(t, err) err = kv1.CAS(context.Background(), "data", func(in interface{}) (out interface{}, retry bool, err error) { var d *data = nil @@ -776,10 +789,6 @@ func TestMultipleCodecs(t *testing.T) { require.NoError(t, err) defer mkv2.Stop() - // We need to register codec to second KV. Normally client does that, but we don't have any client for second KV. - mkv2.RegisterCodec(dataCodec{}) - mkv2.RegisterCodec(distributedCounterCodec{}) - // Join second KV to first one. That will also trigger state transfer. _, err = mkv2.JoinMembers([]string{fmt.Sprintf("127.0.0.1:%d", mkv1.GetListeningPort())}) require.NoError(t, err) From cd7159c3c5e93b50dae5d64d0c7013550106c4e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 20:50:08 +0100 Subject: [PATCH 10/29] Report unknown codec errors as invalid messages. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index fc77f2f1065..e1699275ea5 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -633,6 +633,7 @@ func (m *KV) NotifyMsg(msg []byte) { codec := m.GetCodec(kvPair.GetCodec()) if codec == nil { + m.numberOfInvalidReceivedMessages.Inc() level.Error(util.Logger).Log("msg", "failed to decode received value, unknown codec", "codec", kvPair.GetCodec()) return } From 9a07c27ab43f625b8a8187938ef61da78be7f301 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:07:32 +0100 Subject: [PATCH 11/29] Updated CHANGELOG.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index efce19ece0c..6f201774331 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -32,6 +32,7 @@ * [BUGFIX] Experimental TSDB: fixed `/all_user_stats` and `/api/prom/user_stats` endpoints when using the experimental TSDB blocks storage. #2042 * [BUGFIX] Experimental TSDB: fixed ruler to correctly work with the experimental TSDB blocks storage. #2101 * [BUGFIX] Azure Blob ChunkStore: Fixed issue causing `invalid chunk checksum` errors. #2074 +* [BUGFIX] Experimental Memberlist KV store can now be used in single-binary Cortex. Attempts to use it previously would fail with panic. This change also breaks existing binary protocol used to exchange gossip messages, so this version will not be able to understand gossiped Ring when used in combination with previous version of Cortex. #2016 Cortex 0.4.0 is the last version that can *write* denormalised tokens. Cortex 0.5.0 and above always write normalised tokens. From 3c62bbe2ef34dd301b038efe908f8aa6fde88e0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:23:04 +0100 Subject: [PATCH 12/29] Typos MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6f201774331..a404524ea06 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -32,7 +32,7 @@ * [BUGFIX] Experimental TSDB: fixed `/all_user_stats` and `/api/prom/user_stats` endpoints when using the experimental TSDB blocks storage. #2042 * [BUGFIX] Experimental TSDB: fixed ruler to correctly work with the experimental TSDB blocks storage. #2101 * [BUGFIX] Azure Blob ChunkStore: Fixed issue causing `invalid chunk checksum` errors. #2074 -* [BUGFIX] Experimental Memberlist KV store can now be used in single-binary Cortex. Attempts to use it previously would fail with panic. This change also breaks existing binary protocol used to exchange gossip messages, so this version will not be able to understand gossiped Ring when used in combination with previous version of Cortex. #2016 +* [BUGFIX] Experimental Memberlist KV store can now be used in single-binary Cortex. Attempts to use it previously would fail with panic. This change also breaks existing binary protocol used to exchange gossip messages, so this version will not be able to understand gossiped Ring when used in combination with the previous version of Cortex. #2016 Cortex 0.4.0 is the last version that can *write* denormalised tokens. Cortex 0.5.0 and above always write normalised tokens. From cfc07cdcb25245ec372398f5380c5b1f3b9076a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:26:10 +0100 Subject: [PATCH 13/29] Comments. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/cortex/memberlist_kv.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/pkg/cortex/memberlist_kv.go b/pkg/cortex/memberlist_kv.go index 773efca42b0..521e605cce5 100644 --- a/pkg/cortex/memberlist_kv.go +++ b/pkg/cortex/memberlist_kv.go @@ -6,11 +6,17 @@ import ( "github.com/cortexproject/cortex/pkg/ring/kv/memberlist" ) +// This struct holds state of initialization of memberlist.KV instance. type memberlistKVState struct { - cfg *memberlist.KVConfig + // config used for initialization + cfg *memberlist.KVConfig + + // init function, to avoid multiple initializations. init sync.Once - kv *memberlist.KV - err error + + // state + kv *memberlist.KV + err error } func newMemberlistKVState(cfg *memberlist.KVConfig) *memberlistKVState { From 448e70e17e296e61710d33fafb1c85e47596e1b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:27:10 +0100 Subject: [PATCH 14/29] Fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 2 +- tools/doc-generator/main.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index e1699275ea5..59aa7dc5d9e 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -90,7 +90,7 @@ type KVConfig struct { MetricsNamespace string `yaml:"-"` // Codecs to register. Codecs need to be registered before joining other members. - Codecs []codec.Codec + Codecs []codec.Codec `yaml:"-"` } // RegisterFlags registers flags. diff --git a/tools/doc-generator/main.go b/tools/doc-generator/main.go index 66ac56173e9..11b80d999e3 100644 --- a/tools/doc-generator/main.go +++ b/tools/doc-generator/main.go @@ -114,7 +114,7 @@ var ( }, { name: "memberlist_config", - structType: reflect.TypeOf(memberlist.Config{}), + structType: reflect.TypeOf(memberlist.KVConfig{}), desc: "The memberlist_config configures the Gossip memberlist.", }, { From 55d0f04c4d987f8c357dbd90f0928d835dfed6b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:31:36 +0100 Subject: [PATCH 15/29] Added yaml tag to ignore function. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 7e8de46732c..50304954abc 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -26,7 +26,7 @@ type StoreConfig struct { Etcd etcd.Config `yaml:"etcd,omitempty"` Multi MultiConfig `yaml:"multi,omitempty"` - MemberlistKV func() (*memberlist.KV, error) + MemberlistKV func() (*memberlist.KV, error) `yaml:"-"` } // Config is config for a KVStore currently used by ring and HA tracker, From c9d601160719fdf0bab642215fb3691ea202bc3d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:33:25 +0100 Subject: [PATCH 16/29] Fixed unintended import reorder. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 59aa7dc5d9e..8fb7684ccd5 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -12,12 +12,13 @@ import ( "sync" "time" - "github.com/cortexproject/cortex/pkg/ring/kv/codec" - "github.com/cortexproject/cortex/pkg/util" - "github.com/cortexproject/cortex/pkg/util/flagext" "github.com/go-kit/kit/log/level" "github.com/hashicorp/memberlist" "github.com/prometheus/client_golang/prometheus" + + "github.com/cortexproject/cortex/pkg/ring/kv/codec" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/flagext" ) const ( From b5104148ed72fe30ed8b11440a7339d8e03a9ba5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:34:55 +0100 Subject: [PATCH 17/29] Comments. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 8fb7684ccd5..5952e406280 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -26,11 +26,13 @@ const ( noChangeDetectedRetrySleep = time.Second // how long to sleep after no change was detected in CAS ) +// Client implements kv.Client interface, by using memberlist.KV type Client struct { kv *KV // reference to singleton memberlist-based KV codec codec.Codec } +// NewClient creates new client instance. Supplied codec must already be registered in KV. func NewClient(kv *KV, codec codec.Codec) (*Client, error) { c := kv.GetCodec(codec.CodecID()) if c == nil { From 5133bc815f3613f621324fd842a4912e6eab5dbf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Tue, 21 Jan 2020 21:38:48 +0100 Subject: [PATCH 18/29] Comments. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 5952e406280..a97fb43389d 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -66,7 +66,7 @@ func (c *Client) WatchPrefix(ctx context.Context, prefix string, f func(string, c.kv.WatchPrefix(ctx, prefix, c.codec, f) } -// Config for memberlist-based Client +// KVConfig is a config for memberlist.KV type KVConfig struct { // Memberlist options. NodeName string `yaml:"node_name"` @@ -113,6 +113,8 @@ func (cfg *KVConfig) RegisterFlags(f *flag.FlagSet, prefix string) { cfg.TCPTransport.RegisterFlags(f, prefix) } +// KV implements Key-Value store on top of memberlist library. KV store has API similar to kv.Client, +// except methods also need explicit codec for each operation. type KV struct { cfg KVConfig memberlist *memberlist.Memberlist From 5aae6eb6bb0d8547a468569538a49b5a3c01775f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 22 Jan 2020 08:08:52 +0100 Subject: [PATCH 19/29] Return error MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/client.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index 50304954abc..d2493a2f6bd 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -119,6 +119,9 @@ func createClient(name string, prefix string, cfg StoreConfig, codec codec.Codec return nil, err } client, err = memberlist.NewClient(kv, codec) + if err != nil { + return nil, err + } case "multi": client, err = buildMultiClient(cfg, codec) From fd1fe3f0de96b99d88de05a7fc067c1320dc2b53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 22 Jan 2020 08:57:57 +0100 Subject: [PATCH 20/29] =?UTF-8?q?Updated=20docs=20Signed-off-by:=20Peter?= =?UTF-8?q?=20S=CC=8Ctibrany=CC=81=20?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/configuration/config-file-reference.md | 46 ++++++++------------- 1 file changed, 17 insertions(+), 29 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 3faf24c5b97..ecd5036bcc3 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -102,6 +102,9 @@ runtime_config: # File with the configuration that can be updated in runtime. # CLI flag: -runtime-config.file [file: | default = ""] + +# The memberlist_config configures the Gossip memberlist. +[memberlist_kv: ] ``` ## `server_config` @@ -271,10 +274,6 @@ ha_tracker: # The CLI flags prefix for this block config is: distributor.ha-tracker [etcd: ] - # The memberlist_config configures the Gossip memberlist. - # The CLI flags prefix for this block config is: distributor.ha-tracker - [memberlist: ] - multi: # Primary backend storage used by multi-client. # CLI flag: -distributor.ha-tracker.multi.primary @@ -328,10 +327,6 @@ ring: # The CLI flags prefix for this block config is: distributor.ring [etcd: ] - # The memberlist_config configures the Gossip memberlist. - # The CLI flags prefix for this block config is: distributor.ring - [memberlist: ] - multi: # Primary backend storage used by multi-client. # CLI flag: -distributor.ring.multi.primary @@ -403,9 +398,6 @@ lifecycler: # The etcd_config configures the etcd client. [etcd: ] - # The memberlist_config configures the Gossip memberlist. - [memberlist: ] - multi: # Primary backend storage used by multi-client. # CLI flag: -multi.primary @@ -740,10 +732,6 @@ ring: # The CLI flags prefix for this block config is: ruler.ring [etcd: ] - # The memberlist_config configures the Gossip memberlist. - # The CLI flags prefix for this block config is: ruler.ring - [memberlist: ] - multi: # Primary backend storage used by multi-client. # CLI flag: -ruler.ring.multi.primary @@ -1739,62 +1727,62 @@ The `memberlist_config` configures the Gossip memberlist. ```yaml # Name of the node in memberlist cluster. Defaults to hostname. -# CLI flag: -.memberlist.nodename +# CLI flag: -memberlist.nodename [node_name: | default = ""] # The timeout for establishing a connection with a remote node, and for # read/write operations. Uses memberlist LAN defaults if 0. -# CLI flag: -.memberlist.stream-timeout +# CLI flag: -memberlist.stream-timeout [stream_timeout: | default = 0s] # Multiplication factor used when sending out messages (factor * log(N+1)). -# CLI flag: -.memberlist.retransmit-factor +# CLI flag: -memberlist.retransmit-factor [retransmit_factor: | default = 0] # How often to use pull/push sync. Uses memberlist LAN defaults if 0. -# CLI flag: -.memberlist.pullpush-interval +# CLI flag: -memberlist.pullpush-interval [pull_push_interval: | default = 0s] # How often to gossip. Uses memberlist LAN defaults if 0. -# CLI flag: -.memberlist.gossip-interval +# CLI flag: -memberlist.gossip-interval [gossip_interval: | default = 0s] # How many nodes to gossip to. Uses memberlist LAN defaults if 0. -# CLI flag: -.memberlist.gossip-nodes +# CLI flag: -memberlist.gossip-nodes [gossip_nodes: | default = 0] # Other cluster members to join. Can be specified multiple times. Memberlist # store is EXPERIMENTAL. -# CLI flag: -.memberlist.join +# CLI flag: -memberlist.join [join_members: | default = ] # If this node fails to join memberlist cluster, abort. -# CLI flag: -.memberlist.abort-if-join-fails +# CLI flag: -memberlist.abort-if-join-fails [abort_if_cluster_join_fails: | default = true] # How long to keep LEFT ingesters in the ring. -# CLI flag: -.memberlist.left-ingesters-timeout +# CLI flag: -memberlist.left-ingesters-timeout [left_ingesters_timeout: | default = 5m0s] # Timeout for leaving memberlist cluster. -# CLI flag: -.memberlist.leave-timeout +# CLI flag: -memberlist.leave-timeout [leave_timeout: | default = 5s] # IP address to listen on for gossip messages. Multiple addresses may be # specified. Defaults to 0.0.0.0 -# CLI flag: -.memberlist.bind-addr +# CLI flag: -memberlist.bind-addr [bind_addr: | default = ] # Port to listen on for gossip messages. -# CLI flag: -.memberlist.bind-port +# CLI flag: -memberlist.bind-port [bind_port: | default = 7946] # Timeout used when connecting to other nodes to send packet. -# CLI flag: -.memberlist.packet-dial-timeout +# CLI flag: -memberlist.packet-dial-timeout [packet_dial_timeout: | default = 5s] # Timeout for writing 'packet' data. -# CLI flag: -.memberlist.packet-write-timeout +# CLI flag: -memberlist.packet-write-timeout [packet_write_timeout: | default = 5s] ``` From 1d3aa66b8392e5cd43e564a8a2f732a5a266f581 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 13:20:29 +0100 Subject: [PATCH 21/29] Improved comments in LocalState and MergeRemoteState MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index a97fb43389d..bc2d74c26d0 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -693,12 +693,12 @@ func (m *KV) LocalState(join bool) []byte { m.storeMu.Lock() defer m.storeMu.Unlock() - // For each Key/Value pair in our store, we write: - // [1 byte key length] [key] [4-bytes value length] [value] - - kvPair := KeyValuePair{} + // For each Key/Value pair in our store, we write + // [4-bytes length of KV pair] [KV pair] buf := bytes.Buffer{} + + kvPair := KeyValuePair{} for key, val := range m.store { if val.value == nil { continue @@ -735,6 +735,8 @@ func (m *KV) LocalState(join bool) []byte { // MergeRemoteState is method from Memberlist Delegate interface // // This is 'push' part of push/pull sync. We merge incoming KV store (all keys and values) with ours. +// +// Data is full state of remote KV store, as generated by `LocalState` method (run on another node). func (m *KV) MergeRemoteState(data []byte, join bool) { m.numberOfPushes.Inc() m.totalSizeOfPushes.Add(float64(len(data))) @@ -742,6 +744,8 @@ func (m *KV) MergeRemoteState(data []byte, join bool) { kvPair := KeyValuePair{} var err error + // Data contains individual KV pairs (encoded as protobuf messages), each prefixed with 4 bytes length of KV pair: + // [4-bytes length] [KV pair] [4-bytes length] [KV pair]... for len(data) > 0 { if len(data) < 4 { err = fmt.Errorf("not enough data left for another KV Pair: %d", len(data)) From 1be6245e9e21ba64160d46a6ddd60fb12d19523c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 13:32:21 +0100 Subject: [PATCH 22/29] Allow ruler and distributor use memberlist.KV for their internal rings. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/cortex/modules.go | 2 ++ pkg/ring/kv/client.go | 2 ++ 2 files changed, 4 insertions(+) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 7fd8717494b..9ab956d6ed0 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -214,6 +214,7 @@ func (t *Cortex) initOverrides(cfg *Config) (err error) { func (t *Cortex) initDistributor(cfg *Config) (err error) { cfg.Distributor.DistributorRing.ListenPort = cfg.Server.GRPCListenPort + cfg.Distributor.DistributorRing.KVStore.MemberlistKV = t.memberlistKVState.getMemberlistKV // Check whether the distributor can join the distributors ring, which is // whenever it's not running as an internal dependency (ie. querier or @@ -437,6 +438,7 @@ func (t *Cortex) stopTableManager() error { func (t *Cortex) initRuler(cfg *Config) (err error) { cfg.Ruler.Ring.ListenPort = cfg.Server.GRPCListenPort + cfg.Ruler.Ring.KVStore.MemberlistKV = t.memberlistKVState.getMemberlistKV queryable, engine := querier.New(cfg.Querier, t.distributor, t.querierChunkStore) t.ruler, err = ruler.NewRuler(cfg.Ruler, engine, queryable, t.distributor, prometheus.DefaultRegisterer, util.Logger) diff --git a/pkg/ring/kv/client.go b/pkg/ring/kv/client.go index d2493a2f6bd..1c729fe15df 100644 --- a/pkg/ring/kv/client.go +++ b/pkg/ring/kv/client.go @@ -26,6 +26,8 @@ type StoreConfig struct { Etcd etcd.Config `yaml:"etcd,omitempty"` Multi MultiConfig `yaml:"multi,omitempty"` + // Function that returns memberlist.KV store to use. By using a function, we can delay + // initialization of memberlist.KV until it is actually required. MemberlistKV func() (*memberlist.KV, error) `yaml:"-"` } From de08cc403a9f4241c2540bb179304895643452fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 15:44:59 +0100 Subject: [PATCH 23/29] Added test to verify that Memberlist works in single-binary mode, and all distributors can see all ingesters and their tokens. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- integration/framework/scenario.go | 12 +-- integration/framework/service.go | 4 + ...tegration_memberlist_single_binary_test.go | 89 +++++++++++++++++++ 3 files changed, 99 insertions(+), 6 deletions(-) create mode 100644 integration/integration_memberlist_single_binary_test.go diff --git a/integration/framework/scenario.go b/integration/framework/scenario.go index bafe72336ae..f27f635f4be 100644 --- a/integration/framework/scenario.go +++ b/integration/framework/scenario.go @@ -126,7 +126,7 @@ func (s *Scenario) StartDynamoDB() error { func (s *Scenario) StartDistributor(name string, flags map[string]string, image string) error { if image == "" { - image = getDefaultCortexImage() + image = GetDefaultCortexImage() } return s.StartService(NewService( @@ -150,7 +150,7 @@ func (s *Scenario) StartDistributor(name string, flags map[string]string, image func (s *Scenario) StartQuerier(name string, flags map[string]string, image string) error { if image == "" { - image = getDefaultCortexImage() + image = GetDefaultCortexImage() } return s.StartService(NewService( @@ -173,7 +173,7 @@ func (s *Scenario) StartQuerier(name string, flags map[string]string, image stri func (s *Scenario) StartIngester(name string, flags map[string]string, image string) error { if image == "" { - image = getDefaultCortexImage() + image = GetDefaultCortexImage() } return s.StartService(NewService( @@ -201,7 +201,7 @@ func (s *Scenario) StartIngester(name string, flags map[string]string, image str func (s *Scenario) StartTableManager(name string, flags map[string]string, image string) error { if image == "" { - image = getDefaultCortexImage() + image = GetDefaultCortexImage() } return s.StartService(NewService( @@ -305,8 +305,8 @@ func existDockerNetwork() (bool, error) { return strings.TrimSpace(string(out)) != "", nil } -// getDefaultCortexImage returns the Docker image to use to run Cortex. -func getDefaultCortexImage() string { +// GetDefaultCortexImage returns the Docker image to use to run Cortex. +func GetDefaultCortexImage() string { // Get the cortex image from the CORTEX_IMAGE env variable, // falling back to "quay.io/cortexproject/cortex:latest" if os.Getenv("CORTEX_IMAGE") != "" { diff --git a/integration/framework/service.go b/integration/framework/service.go index bb759b0631b..a0f91b38525 100644 --- a/integration/framework/service.go +++ b/integration/framework/service.go @@ -64,6 +64,10 @@ func NewService( } } +func (s *Service) SetBackoff(cfg util.BackoffConfig) { + s.retryBackoff = util.NewBackoff(context.Background(), cfg) +} + func (s *Service) Start() (err error) { // In case of any error, if the container was already created, we // have to cleanup removing it. We ignore the error of the "docker rm" diff --git a/integration/integration_memberlist_single_binary_test.go b/integration/integration_memberlist_single_binary_test.go new file mode 100644 index 00000000000..5cd8c787091 --- /dev/null +++ b/integration/integration_memberlist_single_binary_test.go @@ -0,0 +1,89 @@ +package main + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/cortexproject/cortex/integration/framework" + "github.com/cortexproject/cortex/pkg/util" +) + +func TestSingleBinaryWithMemberlist(t *testing.T) { + s, err := framework.NewScenario() + require.NoError(t, err) + defer s.Shutdown() + + // Start dependencies + require.NoError(t, s.StartDynamoDB()) + // Look ma, no Consul! + require.NoError(t, s.WaitReady("dynamodb")) + + require.NoError(t, startSingleBinary(s, "cortex-1", "")) + require.NoError(t, startSingleBinary(s, "cortex-2", "cortex-1:8000")) + require.NoError(t, startSingleBinary(s, "cortex-3", "cortex-2:8000")) + + require.NoError(t, s.WaitReady("cortex-1", "cortex-2", "cortex-3")) + + // All three Cortex serves should see each other. + require.NoError(t, s.Service("cortex-1").WaitMetric(80, "memberlist_client_cluster_members_count", 3)) + require.NoError(t, s.Service("cortex-2").WaitMetric(80, "memberlist_client_cluster_members_count", 3)) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "memberlist_client_cluster_members_count", 3)) + + // All Cortex servers should have 512 tokens, altogether 3 * 512 + require.NoError(t, s.Service("cortex-1").WaitMetric(80, "cortex_ring_tokens_total", 3*512)) + require.NoError(t, s.Service("cortex-2").WaitMetric(80, "cortex_ring_tokens_total", 3*512)) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "cortex_ring_tokens_total", 3*512)) + + require.NoError(t, s.StopService("cortex-1")) + require.NoError(t, s.Service("cortex-2").WaitMetric(80, "cortex_ring_tokens_total", 2*512)) + require.NoError(t, s.Service("cortex-2").WaitMetric(80, "memberlist_client_cluster_members_count", 2)) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "cortex_ring_tokens_total", 2*512)) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "memberlist_client_cluster_members_count", 2)) + + require.NoError(t, s.StopService("cortex-2")) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "cortex_ring_tokens_total", 1*512)) + require.NoError(t, s.Service("cortex-3").WaitMetric(80, "memberlist_client_cluster_members_count", 1)) + + require.NoError(t, s.StopService("cortex-3")) +} + +func startSingleBinary(s *framework.Scenario, name string, join string) error { + flags := map[string]string{ + "-target": "all", // single-binary mode + "-log.level": "warn", + "-ingester.final-sleep": "0s", + "-ingester.join-after": "0s", // join quickly + "-ingester.min-ready-duration": "0s", + "-ingester.concurrent-flushes": "10", + "-ingester.max-transfer-retries": "0", // disable + "-ingester.num-tokens": "512", + "-ingester.observe-period": "5s", // to avoid conflicts in tokens + "-ring.store": "memberlist", + "-memberlist.bind-port": "8000", + } + + if join != "" { + flags["-memberlist.join"] = join + } + + serv := framework.NewService( + name, + framework.GetDefaultCortexImage(), + framework.NetworkName, + []int{80, 8000}, + nil, + framework.NewCommandWithoutEntrypoint("cortex", framework.BuildArgs(framework.MergeFlags(ChunksStorage, flags))...), + framework.NewReadinessProbe(80, "/ready", 204), + ) + + backOff := util.BackoffConfig{ + MinBackoff: 100 * time.Millisecond, + MaxBackoff: 500 * time.Millisecond, // bump max backoff... things take little longer with memberlist + MaxRetries: 100, + } + + serv.SetBackoff(backOff) + return s.StartService(serv) +} From 7f05293d25dd7619980c773271e7b1166650988f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 15:51:03 +0100 Subject: [PATCH 24/29] Updated CHANGELOG.md, moved to CHANGE because of protocol break. Added note on how to upgrade. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a404524ea06..9df7f7f68c4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * [CHANGE] Moved `--store.min-chunk-age` to the Querier config as `--querier.query-store-after`, allowing the store to be skipped during query time if the metrics wouldn't be found. The YAML config option `ingestermaxquerylookback` has been renamed to `query_ingesters_within` to match its CLI flag. #1893 * `--store.min-chunk-age` has been removed * `--querier.query-store-after` has been added in it's place. +* [CHANGE] Experimental Memberlist KV store can now be used in single-binary Cortex. Attempts to use it previously would fail with panic. This change also breaks existing binary protocol used to exchange gossip messages, so this version will not be able to understand gossiped Ring when used in combination with the previous version of Cortex. Easiest way to upgrade is to shutdown old Cortex installation, and restart it with new version. Incremental rollout works too, but with reduced functionality until all components run the same version. #2016 * [FEATURE] Added user sub rings to distribute users to a subset of ingesters. #1947 * `--experimental.distributor.user-subring-size` * [FEATURE] Added flag `-experimental.ruler.enable-api` to enable the ruler api which implements the Prometheus API `/api/v1/rules` and `/api/v1/alerts` endpoints under the configured `-http.prefix`. #1999 @@ -32,7 +33,6 @@ * [BUGFIX] Experimental TSDB: fixed `/all_user_stats` and `/api/prom/user_stats` endpoints when using the experimental TSDB blocks storage. #2042 * [BUGFIX] Experimental TSDB: fixed ruler to correctly work with the experimental TSDB blocks storage. #2101 * [BUGFIX] Azure Blob ChunkStore: Fixed issue causing `invalid chunk checksum` errors. #2074 -* [BUGFIX] Experimental Memberlist KV store can now be used in single-binary Cortex. Attempts to use it previously would fail with panic. This change also breaks existing binary protocol used to exchange gossip messages, so this version will not be able to understand gossiped Ring when used in combination with the previous version of Cortex. #2016 Cortex 0.4.0 is the last version that can *write* denormalised tokens. Cortex 0.5.0 and above always write normalised tokens. From 91fe0635183a324334be1731431e84e0d6976c2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 15:53:46 +0100 Subject: [PATCH 25/29] Use "memberlist" in YAML config. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/cortex/cortex.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index da5da232414..78989ff3083 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -85,7 +85,7 @@ type Config struct { ConfigStore config_client.Config `yaml:"config_store,omitempty"` Alertmanager alertmanager.MultitenantAlertmanagerConfig `yaml:"alertmanager,omitempty"` RuntimeConfig runtimeconfig.ManagerConfig `yaml:"runtime_config,omitempty"` - MemberlistKV memberlist.KVConfig `yaml:"memberlist_kv"` + MemberlistKV memberlist.KVConfig `yaml:"memberlist"` } // RegisterFlags registers flag. From bca87c253b1b37efc26899ec77dfda61b1485404 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 15:54:21 +0100 Subject: [PATCH 26/29] Updated comment. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/codec/codec.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ring/kv/codec/codec.go b/pkg/ring/kv/codec/codec.go index 3aab3c9afd3..14acecb70b4 100644 --- a/pkg/ring/kv/codec/codec.go +++ b/pkg/ring/kv/codec/codec.go @@ -10,7 +10,7 @@ type Codec interface { Decode([]byte) (interface{}, error) Encode(interface{}) ([]byte, error) - // Short ID to communicate what codec should be used to decode the value. + // CodecID is a short identifier to communicate what codec should be used to decode the value. // Once in use, this should be stable to avoid confusing other clients. CodecID() string } From 7dbf90f70596a52d08450c04eb95879599509ca8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 15:57:14 +0100 Subject: [PATCH 27/29] Updated comments and messages. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/ring/kv/memberlist/memberlist_client.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index bc2d74c26d0..75e3f31e4dc 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -264,9 +264,9 @@ func NewKV(cfg KVConfig) (*KV, error) { } if err != nil { - level.Error(util.Logger).Log("msg", "Failed to join memberlist cluster", "err", err) + level.Error(util.Logger).Log("msg", "failed to join memberlist cluster", "err", err) } else { - level.Info(util.Logger).Log("msg", "Joined memberlist cluster", "reached_nodes", reached) + level.Info(util.Logger).Log("msg", "joined memberlist cluster", "reached_nodes", reached) } } @@ -625,13 +625,13 @@ func (m *KV) NotifyMsg(msg []byte) { kvPair := KeyValuePair{} err := kvPair.Unmarshal(msg) if err != nil { - level.Warn(util.Logger).Log("msg", "Failed to unmarshal received KV Pair", "err", err) + level.Warn(util.Logger).Log("msg", "failed to unmarshal received KV Pair", "err", err) m.numberOfInvalidReceivedMessages.Inc() return } if len(kvPair.Key) == 0 { - level.Warn(util.Logger).Log("msg", "Invalid KV Pair, empty key") + level.Warn(util.Logger).Log("msg", "received an invalid KV Pair (empty key)") m.numberOfInvalidReceivedMessages.Inc() return } @@ -694,7 +694,7 @@ func (m *KV) LocalState(join bool) []byte { defer m.storeMu.Unlock() // For each Key/Value pair in our store, we write - // [4-bytes length of KV pair] [KV pair] + // [4-bytes length of marshalled KV pair] [marshalled KV pair] buf := bytes.Buffer{} @@ -745,7 +745,7 @@ func (m *KV) MergeRemoteState(data []byte, join bool) { var err error // Data contains individual KV pairs (encoded as protobuf messages), each prefixed with 4 bytes length of KV pair: - // [4-bytes length] [KV pair] [4-bytes length] [KV pair]... + // [4-bytes length of marshalled KV pair] [marshalled KV pair] [4-bytes length] [KV pair]... for len(data) > 0 { if len(data) < 4 { err = fmt.Errorf("not enough data left for another KV Pair: %d", len(data)) From 11ff1a5400955a4fdf4204830be7892c7b2c4e81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 16:04:55 +0100 Subject: [PATCH 28/29] Fixed config. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- docs/configuration/config-file-reference.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index ecd5036bcc3..40a2b0ef221 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -104,7 +104,7 @@ runtime_config: [file: | default = ""] # The memberlist_config configures the Gossip memberlist. -[memberlist_kv: ] +[memberlist: ] ``` ## `server_config` From fb4f21147cf33fcff2a6c1dfbdc39900c7541743 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Peter=20S=CC=8Ctibrany=CC=81?= Date: Wed, 12 Feb 2020 16:47:21 +0100 Subject: [PATCH 29/29] Compactor can now also use memberlist KV store. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Peter Štibraný --- pkg/cortex/modules.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 9ab956d6ed0..2a823e5b03a 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -499,6 +499,7 @@ func (t *Cortex) stopAlertmanager() error { func (t *Cortex) initCompactor(cfg *Config) (err error) { cfg.Compactor.ShardingRing.ListenPort = cfg.Server.GRPCListenPort + cfg.Compactor.ShardingRing.KVStore.MemberlistKV = t.memberlistKVState.getMemberlistKV t.compactor, err = compactor.NewCompactor(cfg.Compactor, cfg.TSDB, util.Logger, prometheus.DefaultRegisterer) if err != nil {