-
Notifications
You must be signed in to change notification settings - Fork 867
feat(shard distributor): Persist Shard-Level Metrics With Guarded Updates for Load Balancing #7354
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from 9 commits
2de12d8
5d95067
6e57536
595d320
d9ba54d
32d2ecd
b624a00
aad7b2e
6360f8a
1536d0a
f316fbf
4524da9
126f725
cc53f68
733bbcb
6816b8e
f97e0cf
513e88c
9833525
0332fe5
d5a13d9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -11,6 +11,7 @@ const ( | |
| ExecutorReportedShardsKey = "reported_shards" | ||
| ExecutorAssignedStateKey = "assigned_state" | ||
| ShardAssignedKey = "assigned" | ||
| ShardMetricsKey = "metrics" | ||
| ) | ||
|
|
||
| var validKeyTypes = []string{ | ||
|
|
@@ -57,3 +58,27 @@ func ParseExecutorKey(prefix string, namespace, key string) (executorID, keyType | |
| } | ||
| return parts[0], parts[1], nil | ||
| } | ||
|
|
||
| func BuildShardPrefix(prefix string, namespace string) string { | ||
| return fmt.Sprintf("%s/shards/", BuildNamespacePrefix(prefix, namespace)) | ||
| } | ||
|
|
||
| func BuildShardKey(prefix string, namespace, shardID, keyType string) (string, error) { | ||
| if keyType != ShardAssignedKey && keyType != ShardMetricsKey { | ||
|
||
| return "", fmt.Errorf("invalid shard key type: %s", keyType) | ||
| } | ||
| return fmt.Sprintf("%s%s/%s", BuildShardPrefix(prefix, namespace), shardID, keyType), nil | ||
| } | ||
|
|
||
| func ParseShardKey(prefix string, namespace, key string) (shardID, keyType string, err error) { | ||
| prefix = BuildShardPrefix(prefix, namespace) | ||
| if !strings.HasPrefix(key, prefix) { | ||
| return "", "", fmt.Errorf("key '%s' does not have expected prefix '%s'", key, prefix) | ||
| } | ||
| remainder := strings.TrimPrefix(key, prefix) | ||
| parts := strings.Split(remainder, "/") | ||
| if len(parts) != 2 { | ||
| return "", "", fmt.Errorf("unexpected shard key format: %s", key) | ||
| } | ||
| return parts[0], parts[1], nil | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -34,6 +34,18 @@ type executorStoreImpl struct { | |
| shardCache *shardcache.ShardToExecutorCache | ||
| } | ||
|
|
||
| // shardMetricsUpdate tracks the etcd key, revision, and metrics used to update a shard | ||
| // after the main transaction in AssignShards for exec state. | ||
| // Retains metrics to safely merge concurrent updates before retrying. | ||
| type shardMetricsUpdate struct { | ||
|
||
| key string | ||
| shardID string | ||
| metrics store.ShardMetrics | ||
| modRevision int64 | ||
| desiredLastMove int64 // intended LastMoveTime for this update | ||
| defaultLastUpdate int64 | ||
|
||
| } | ||
|
|
||
| // ExecutorStoreParams defines the dependencies for the etcd store, for use with fx. | ||
| type ExecutorStoreParams struct { | ||
| fx.In | ||
|
|
@@ -201,6 +213,7 @@ func (s *executorStoreImpl) GetHeartbeat(ctx context.Context, namespace string, | |
| func (s *executorStoreImpl) GetState(ctx context.Context, namespace string) (*store.NamespaceState, error) { | ||
| heartbeatStates := make(map[string]store.HeartbeatState) | ||
| assignedStates := make(map[string]store.AssignedState) | ||
| shardMetrics := make(map[string]store.ShardMetrics) | ||
|
|
||
| executorPrefix := etcdkeys.BuildExecutorPrefix(s.prefix, namespace) | ||
| resp, err := s.client.Get(ctx, executorPrefix, clientv3.WithPrefix()) | ||
|
|
@@ -242,8 +255,30 @@ func (s *executorStoreImpl) GetState(ctx context.Context, namespace string) (*st | |
| assignedStates[executorID] = assigned | ||
| } | ||
|
|
||
| // Fetch shard-level metrics stored under shard namespace keys. | ||
| shardPrefix := etcdkeys.BuildShardPrefix(s.prefix, namespace) | ||
| shardResp, err := s.client.Get(ctx, shardPrefix, clientv3.WithPrefix()) | ||
| if err != nil { | ||
| return nil, fmt.Errorf("get shard data: %w", err) | ||
| } | ||
| for _, kv := range shardResp.Kvs { | ||
| shardID, shardKeyType, err := etcdkeys.ParseShardKey(s.prefix, namespace, string(kv.Key)) | ||
| if err != nil { | ||
| continue | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we don't want to abort metric emission , we should still log an error such that we have evidence that something is not working as expected. |
||
| if shardKeyType != etcdkeys.ShardMetricsKey { | ||
| continue | ||
| } | ||
| var shardMetric store.ShardMetrics | ||
| if err := json.Unmarshal(kv.Value, &shardMetric); err != nil { | ||
| continue | ||
| } | ||
| shardMetrics[shardID] = shardMetric | ||
| } | ||
|
|
||
| return &store.NamespaceState{ | ||
| Executors: heartbeatStates, | ||
| ShardMetrics: shardMetrics, | ||
| ShardAssignments: assignedStates, | ||
| GlobalRevision: resp.Header.Revision, | ||
| }, nil | ||
|
|
@@ -294,6 +329,75 @@ func (s *executorStoreImpl) AssignShards(ctx context.Context, namespace string, | |
| var ops []clientv3.Op | ||
| var comparisons []clientv3.Cmp | ||
|
|
||
| // Compute shard moves to update last_move_time metrics when ownership changes. | ||
| // Read current assignments for the namespace and compare with the new state. | ||
| // Concurrent changes will be caught by the revision comparisons later. | ||
| currentAssignments := make(map[string]string) // shardID -> executorID | ||
|
||
| executorPrefix := etcdkeys.BuildExecutorPrefix(s.prefix, namespace) | ||
| resp, err := s.client.Get(ctx, executorPrefix, clientv3.WithPrefix()) | ||
| if err != nil { | ||
| return fmt.Errorf("get current assignments: %w", err) | ||
| } | ||
| for _, kv := range resp.Kvs { | ||
| executorID, keyType, keyErr := etcdkeys.ParseExecutorKey(s.prefix, namespace, string(kv.Key)) | ||
| if keyErr != nil || keyType != etcdkeys.ExecutorAssignedStateKey { | ||
| continue | ||
| } | ||
| var state store.AssignedState | ||
| if err := json.Unmarshal(kv.Value, &state); err != nil { | ||
| return fmt.Errorf("unmarshal current assigned state: %w", err) | ||
| } | ||
| for shardID := range state.AssignedShards { | ||
| currentAssignments[shardID] = executorID | ||
| } | ||
| } | ||
|
|
||
| // Build new owner map and detect moved shards. | ||
| newAssignments := make(map[string]string) | ||
| for executorID, state := range request.NewState.ShardAssignments { | ||
| for shardID := range state.AssignedShards { | ||
| newAssignments[shardID] = executorID | ||
| } | ||
| } | ||
| now := time.Now().Unix() | ||
|
||
| // Collect metric updates now so we can apply them after committing the main transaction. | ||
|
||
| var metricsUpdates []shardMetricsUpdate | ||
| for shardID, newOwner := range newAssignments { | ||
| if oldOwner, ok := currentAssignments[shardID]; ok && oldOwner == newOwner { | ||
| continue | ||
| } | ||
| // For a new or moved shard, update last_move_time while keeping existing metrics if available. | ||
| shardMetricsKey, err := etcdkeys.BuildShardKey(s.prefix, namespace, shardID, etcdkeys.ShardMetricsKey) | ||
| if err != nil { | ||
| return fmt.Errorf("build shard metrics key: %w", err) | ||
| } | ||
| var shardMetrics store.ShardMetrics | ||
| metricsModRevision := int64(0) | ||
| metricsResp, err := s.client.Get(ctx, shardMetricsKey) | ||
| if err != nil { | ||
| return fmt.Errorf("get shard metrics: %w", err) | ||
| } | ||
| if len(metricsResp.Kvs) > 0 { | ||
| metricsModRevision = metricsResp.Kvs[0].ModRevision | ||
| if err := json.Unmarshal(metricsResp.Kvs[0].Value, &shardMetrics); err != nil { | ||
| return fmt.Errorf("unmarshal shard metrics: %w", err) | ||
| } | ||
| } else { | ||
| shardMetrics.SmoothedLoad = 0 | ||
| shardMetrics.LastUpdateTime = now | ||
| } | ||
| // Do not set LastMoveTime here, it will be applied later to avoid overwriting | ||
| // a newer timestamp if a concurrent rebalance has already updated it. | ||
| metricsUpdates = append(metricsUpdates, shardMetricsUpdate{ | ||
| key: shardMetricsKey, | ||
| shardID: shardID, | ||
| metrics: shardMetrics, | ||
| modRevision: metricsModRevision, | ||
| desiredLastMove: now, | ||
| defaultLastUpdate: shardMetrics.LastUpdateTime, | ||
| }) | ||
| } | ||
|
|
||
| // 1. Prepare operations to update executor states and shard ownership, | ||
| // and comparisons to check for concurrent modifications. | ||
| for executorID, state := range request.NewState.ShardAssignments { | ||
|
|
@@ -357,9 +461,81 @@ func (s *executorStoreImpl) AssignShards(ctx context.Context, namespace string, | |
| return fmt.Errorf("%w: transaction failed, a shard may have been concurrently assigned", store.ErrVersionConflict) | ||
| } | ||
|
|
||
| // Apply shard metrics updates outside the main transaction to stay within etcd's max operations per txn. | ||
| s.applyShardMetricsUpdates(ctx, namespace, metricsUpdates) | ||
|
|
||
| return nil | ||
| } | ||
|
|
||
| // applyShardMetricsUpdates updates shard metrics (like last_move_time) after AssignShards. | ||
| // Decided to run these writes outside the primary transaction | ||
| // so we are less likely to exceed etcd's max txn-op threshold (128?), and we retry | ||
| // logs failures instead of failing the main assignment. | ||
| func (s *executorStoreImpl) applyShardMetricsUpdates(ctx context.Context, namespace string, updates []shardMetricsUpdate) { | ||
| for i := range updates { | ||
| update := &updates[i] | ||
|
|
||
| for { | ||
|
||
| // If a newer rebalance already set a later LastMoveTime, there's nothing left for this iteration. | ||
| if update.metrics.LastMoveTime >= update.desiredLastMove { | ||
| break | ||
| } | ||
|
|
||
| update.metrics.LastMoveTime = update.desiredLastMove | ||
|
|
||
| payload, err := json.Marshal(update.metrics) | ||
| if err != nil { | ||
| // Log and move on. failing metrics formatting should not invalidate the finished assignment. | ||
| s.logger.Warn("failed to marshal shard metrics after assignment", tag.ShardNamespace(namespace), tag.ShardKey(update.shardID), tag.Error(err)) | ||
| break | ||
| } | ||
|
|
||
| txnResp, err := s.client.Txn(ctx). | ||
| If(clientv3.Compare(clientv3.ModRevision(update.key), "=", update.modRevision)). | ||
| Then(clientv3.OpPut(update.key, string(payload))). | ||
| Commit() | ||
| if err != nil { | ||
| // log and abandon this shard rather than propagating an error after assignments commit. | ||
| s.logger.Warn("failed to commit shard metrics update after assignment", tag.ShardNamespace(namespace), tag.ShardKey(update.shardID), tag.Error(err)) | ||
| break | ||
| } | ||
|
|
||
| if txnResp.Succeeded { | ||
| break | ||
| } | ||
|
|
||
| if ctx.Err() != nil { | ||
| s.logger.Warn("context canceled while updating shard metrics", tag.ShardNamespace(namespace), tag.ShardKey(update.shardID), tag.Error(ctx.Err())) | ||
| return | ||
| } | ||
|
|
||
| // Another writer beat us. pull the latest metrics so we can merge their view and retry. | ||
| metricsResp, err := s.client.Get(ctx, update.key) | ||
| if err != nil { | ||
| // Unable to observe the conflicting write, so we skip this shard and keep the assignment result. | ||
| s.logger.Warn("failed to refresh shard metrics after compare conflict", tag.ShardNamespace(namespace), tag.ShardKey(update.shardID), tag.Error(err)) | ||
| break | ||
| } | ||
|
|
||
| update.modRevision = 0 | ||
| if len(metricsResp.Kvs) > 0 { | ||
| update.modRevision = metricsResp.Kvs[0].ModRevision | ||
| if err := json.Unmarshal(metricsResp.Kvs[0].Value, &update.metrics); err != nil { | ||
| // If the value is corrupt we cannot safely merge, so we abandon this shard's metrics update. | ||
| s.logger.Warn("failed to unmarshal shard metrics after compare conflict", tag.ShardNamespace(namespace), tag.ShardKey(update.shardID), tag.Error(err)) | ||
| break | ||
| } | ||
| } else { | ||
| update.metrics = store.ShardMetrics{ | ||
| SmoothedLoad: 0, | ||
| LastUpdateTime: update.defaultLastUpdate, | ||
|
||
| } | ||
| update.modRevision = 0 | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| func (s *executorStoreImpl) AssignShard(ctx context.Context, namespace, shardID, executorID string) error { | ||
| assignedState, err := etcdkeys.BuildExecutorKey(s.prefix, namespace, executorID, etcdkeys.ExecutorAssignedStateKey) | ||
| if err != nil { | ||
|
|
@@ -369,16 +545,21 @@ func (s *executorStoreImpl) AssignShard(ctx context.Context, namespace, shardID, | |
| if err != nil { | ||
| return fmt.Errorf("build executor status key: %w", err) | ||
| } | ||
| shardMetricsKey, err := etcdkeys.BuildShardKey(s.prefix, namespace, shardID, etcdkeys.ShardMetricsKey) | ||
| if err != nil { | ||
| return fmt.Errorf("build shard metrics key: %w", err) | ||
| } | ||
|
|
||
| // Use a read-modify-write loop to handle concurrent updates safely. | ||
| for { | ||
| // 1. Get the current assigned state of the executor. | ||
| // 1. Get the current assigned state of the executor and prepare the shard metrics. | ||
| resp, err := s.client.Get(ctx, assignedState) | ||
| if err != nil { | ||
| return fmt.Errorf("get executor state: %w", err) | ||
| } | ||
|
|
||
| var state store.AssignedState | ||
| var shardMetrics store.ShardMetrics | ||
| modRevision := int64(0) // A revision of 0 means the key doesn't exist yet. | ||
|
|
||
| if len(resp.Kvs) > 0 { | ||
|
|
@@ -393,6 +574,28 @@ func (s *executorStoreImpl) AssignShard(ctx context.Context, namespace, shardID, | |
| state.AssignedShards = make(map[string]*types.ShardAssignment) | ||
| } | ||
|
|
||
| metricsResp, err := s.client.Get(ctx, shardMetricsKey) | ||
| if err != nil { | ||
| return fmt.Errorf("get shard metrics: %w", err) | ||
| } | ||
| now := time.Now().Unix() | ||
| metricsModRevision := int64(0) | ||
| if len(metricsResp.Kvs) > 0 { | ||
| metricsModRevision = metricsResp.Kvs[0].ModRevision | ||
| if err := json.Unmarshal(metricsResp.Kvs[0].Value, &shardMetrics); err != nil { | ||
| return fmt.Errorf("unmarshal shard metrics: %w", err) | ||
| } | ||
| // Metrics already exist, update the last move time. | ||
| // This can happen if the shard was previously assigned to an executor, and a lookup happens after the executor is deleted, | ||
| // AssignShard is then called to assign the shard to a new executor. | ||
| shardMetrics.LastMoveTime = now | ||
| } else { | ||
| // Metrics don't exist, initialize them. | ||
| shardMetrics.SmoothedLoad = 0 | ||
| shardMetrics.LastUpdateTime = now | ||
| shardMetrics.LastMoveTime = now | ||
| } | ||
|
|
||
| // 2. Modify the state in memory, adding the new shard if it's not already there. | ||
| if _, alreadyAssigned := state.AssignedShards[shardID]; !alreadyAssigned { | ||
| state.AssignedShards[shardID] = &types.ShardAssignment{Status: types.AssignmentStatusREADY} | ||
|
|
@@ -403,13 +606,19 @@ func (s *executorStoreImpl) AssignShard(ctx context.Context, namespace, shardID, | |
| return fmt.Errorf("marshal new assigned state: %w", err) | ||
| } | ||
|
|
||
| newMetricsValue, err := json.Marshal(shardMetrics) | ||
| if err != nil { | ||
| return fmt.Errorf("marshal new shard metrics: %w", err) | ||
| } | ||
|
|
||
| var comparisons []clientv3.Cmp | ||
|
|
||
| // 3. Prepare and commit the transaction with three atomic checks. | ||
| // 3. Prepare and commit the transaction with four atomic checks. | ||
| // a) Check that the executor's status is ACTIVE. | ||
| comparisons = append(comparisons, clientv3.Compare(clientv3.Value(statusKey), "=", _executorStatusRunningJSON)) | ||
| // b) Check that the assigned_state key hasn't been changed by another process. | ||
| // b) Check that neither the assigned_state nor shard metrics were modified concurrently. | ||
| comparisons = append(comparisons, clientv3.Compare(clientv3.ModRevision(assignedState), "=", modRevision)) | ||
| comparisons = append(comparisons, clientv3.Compare(clientv3.ModRevision(shardMetricsKey), "=", metricsModRevision)) | ||
| // c) Check that the cache is up to date. | ||
| cmp, err := s.shardCache.GetExecutorModRevisionCmp(namespace) | ||
| if err != nil { | ||
|
|
@@ -428,7 +637,10 @@ func (s *executorStoreImpl) AssignShard(ctx context.Context, namespace, shardID, | |
|
|
||
| txnResp, err := s.client.Txn(ctx). | ||
| If(comparisons...). | ||
| Then(clientv3.OpPut(assignedState, string(newStateValue))). | ||
| Then( | ||
| clientv3.OpPut(assignedState, string(newStateValue)), | ||
| clientv3.OpPut(shardMetricsKey, string(newMetricsValue)), | ||
| ). | ||
| Commit() | ||
|
|
||
| if err != nil { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We need to have tests for BuildShardPrefix, BuildShardKey and ParseShardKey :)