diff --git a/internal/agent/agent.go b/internal/agent/agent.go index d1fb97220..1be0a85f3 100644 --- a/internal/agent/agent.go +++ b/internal/agent/agent.go @@ -715,8 +715,8 @@ func (s *BuiltInItemValue) SetValueCounter(value float64, count float64) { s.value.AddValueCounter(value, count) } -func (s *Agent) shard(key *data_model.Key, metricInfo *format.MetricMetaValue) (shardID uint32, newStrategy bool, weightMul int, legacyKeyHash uint64) { - return sharding.Shard(key, metricInfo, s.NumShards(), s.shardByMetricCount, s.newShardingByName.Load()) +func (s *Agent) shard(key *data_model.Key, metricInfo *format.MetricMetaValue, scratch *[]byte) (shardID uint32, newStrategy bool, weightMul int, legacyKeyHash uint64) { + return sharding.Shard(key, metricInfo, s.NumShards(), s.shardByMetricCount, s.newShardingByName.Load(), scratch) } // Do not create too many. ShardReplicas will iterate through values before flushing bucket @@ -750,7 +750,7 @@ func (s *Agent) ApplyMetric(m tlstatshouse.MetricBytes, h data_model.MappedMetri h.InvalidString, 1) return } - shardId, newStrategy, weightMul, resolutionHash := s.shard(&h.Key, h.MetricMeta) + shardId, newStrategy, weightMul, resolutionHash := s.shard(&h.Key, h.MetricMeta, scratch) if shardId >= uint32(len(s.Shards)) { s.AddCounter(0, format.BuiltinMetricMetaIngestionStatus, []int32{h.Key.Tags[0], h.Key.Metric, format.TagValueIDSrcIngestionStatusErrShardingFailed, 0}, @@ -866,7 +866,7 @@ func (s *Agent) AddCounterHostAERA(t uint32, metricInfo *format.MetricMetaValue, key.Tags[format.RouteTag] = aera.Route key.Tags[format.BuildArchTag] = aera.BuildArch } - shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo) + shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo, nil) // resolutionHash will be 0 for built-in metrics, we are OK with this shard := s.Shards[shardId] shard.AddCounterHost(&key, resolutionHash, count, hostTag, metricInfo, weightMul) @@ -894,7 +894,7 @@ func (s *Agent) AddCounterHostStringBytesAERA(t uint32, metricInfo *format.Metri key.Tags[format.RouteTag] = aera.Route key.Tags[format.BuildArchTag] = aera.BuildArch } - shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo) + shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo, nil) // resolutionHash will be 0 for built-in metrics, we are OK with this shard := s.Shards[shardId] shard.AddCounterHostStringBytes(&key, resolutionHash, data_model.TagUnionBytes{S: str, I: 0}, count, hostTag, metricInfo, weightMul) @@ -925,7 +925,7 @@ func (s *Agent) AddValueCounterHostAERA(t uint32, metricInfo *format.MetricMetaV key.Tags[format.RouteTag] = aera.Route key.Tags[format.BuildArchTag] = aera.BuildArch } - shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo) + shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo, nil) // resolutionHash will be 0 for built-in metrics, we are OK with this shard := s.Shards[shardId] shard.AddValueCounterHost(&key, resolutionHash, value, counter, hostTag, metricInfo, weightMul) @@ -952,7 +952,7 @@ func (s *Agent) AddValueCounterStringHostAERA(t uint32, metricInfo *format.Metri key.Tags[format.RouteTag] = aera.Route key.Tags[format.BuildArchTag] = aera.BuildArch } - shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo) + shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo, nil) // resolutionHash will be 0 for built-in metrics, we are OK with this shard := s.Shards[shardId] shard.AddValueCounterStringHost(&key, resolutionHash, topValue, value, counter, hostTag, metricInfo, weightMul) @@ -969,7 +969,7 @@ func (s *Agent) MergeItemValue(t uint32, metricInfo *format.MetricMetaValue, tag key.Tags[format.AggShardTag] = s.AggregatorShardKey key.Tags[format.AggReplicaTag] = s.AggregatorReplicaKey } - shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo) + shardId, _, weightMul, resolutionHash := s.shard(&key, metricInfo, nil) // resolutionHash will be 0 for built-in metrics, we are OK with this shard := s.Shards[shardId] shard.MergeItemValue(&key, resolutionHash, item, metricInfo, weightMul) diff --git a/internal/agent/agent_mapping.go b/internal/agent/agent_mapping.go index 8a69aff73..1f1417f66 100644 --- a/internal/agent/agent_mapping.go +++ b/internal/agent/agent_mapping.go @@ -49,9 +49,8 @@ func (s *Agent) mapAllTags(h *data_model.MappedMetricHeader, metric *tlstatshous // We could arguably call h.SetKey, but there is very little difference in semantic to care continue } - if tagMeta.Index+1 < format.MaxTags { // TODO - remove after NewMaxTags - h.SetTag(tagMeta.Index+1, hi, tagIDKey+1) // last tag is never Raw64, checked by RestoreCachedInfo - } + h.SetTag(tagMeta.Index+1, hi, tagIDKey+1) // last tag is never Raw64, checked by RestoreCachedInfo + tagValue.I = lo case tagMeta.Raw: id, ok := format.ContainsRawTagValue(mem.B(v.Value)) diff --git a/internal/agent/agent_test.go b/internal/agent/agent_test.go index 249bd8d6a..494c9406d 100644 --- a/internal/agent/agent_test.go +++ b/internal/agent/agent_test.go @@ -66,17 +66,6 @@ func Benchmark_Original_Marshal(b *testing.B) { } } -func Benchmark_Hash(b *testing.B) { - var k data_model.Key - var result uint64 - b.ReportAllocs() - for i := 0; i < b.N; i++ { - k.Tags[14]++ - k.Tags[0] = int32(i) - result += k.Hash() - } -} - // cpu: 13th Gen Intel(R) Core(TM) i7-1360P // Benchmark_XXHash-16 10919467 102.8 ns/op 0 B/op 0 allocs/op func Benchmark_XXHash(b *testing.B) { diff --git a/internal/aggregator/aggregator_insert.go b/internal/aggregator/aggregator_insert.go index 8de50cf05..b6da408a0 100644 --- a/internal/aggregator/aggregator_insert.go +++ b/internal/aggregator/aggregator_insert.go @@ -28,16 +28,18 @@ import ( "github.com/vkcom/statshouse/internal/vkgo/srvfunc" ) +const legacyMaxTags = 16 + func getTableDesc(v3Format bool) string { if v3Format { - keysFieldsNamesVec := make([]string, format.NewMaxTags) - for i := 0; i < format.NewMaxTags; i++ { + keysFieldsNamesVec := make([]string, format.MaxTags) + for i := 0; i < format.MaxTags; i++ { keysFieldsNamesVec[i] = fmt.Sprintf(`tag%d,stag%d`, i, i) } return `statshouse_v3_incoming(index_type,metric,time,` + strings.Join(keysFieldsNamesVec, `,`) + `,count,min,max,sum,sumsquare,percentiles,uniq_state,min_host_legacy,max_host_legacy,min_host,max_host)` } - keysFieldsNamesVec := make([]string, format.MaxTags) - for i := 0; i < format.MaxTags; i++ { + keysFieldsNamesVec := make([]string, legacyMaxTags) + for i := 0; i < legacyMaxTags; i++ { keysFieldsNamesVec[i] = fmt.Sprintf(`key%d`, i) } return `statshouse_value_incoming_prekey3(metric,prekey,prekey_set,time,` + strings.Join(keysFieldsNamesVec, `,`) + `,count,min,max,sum,sumsquare,percentiles,uniq_state,skey,min_host,max_host)` @@ -142,8 +144,7 @@ func appendKeys(res []byte, k *data_model.Key, metricCache *metricIndexCache, v3 res = append(res, byte(0)) } res = binary.LittleEndian.AppendUint32(res, k.Timestamp) - tagsN := format.MaxTags - for ki := 0; ki < tagsN; ki++ { + for ki := 0; ki < legacyMaxTags; ki++ { res = appendTag(res, uint32(k.Tags[ki])) } return res @@ -151,11 +152,6 @@ func appendKeys(res []byte, k *data_model.Key, metricCache *metricIndexCache, v3 func appendKeysNewFormat(res []byte, k *data_model.Key, metricCache *metricIndexCache, top data_model.TagUnion, bufferedInsert bool) []byte { appendTag := func(res []byte, k *data_model.Key, i int) []byte { - if i >= len(k.Tags) { // temporary while we in transition between 16 and 48 tags - res = binary.LittleEndian.AppendUint32(res, 0) - res = rowbinary.AppendString(res, "") - return res - } if len(k.GetSTag(i)) > 0 { res = binary.LittleEndian.AppendUint32(res, 0) res = rowbinary.AppendString(res, k.GetSTag(i)) @@ -174,7 +170,7 @@ func appendKeysNewFormat(res []byte, k *data_model.Key, metricCache *metricIndex // TODO write pretags _ = metricCache res = binary.LittleEndian.AppendUint32(res, k.Timestamp) - for ki := 0; ki < format.NewMaxTags; ki++ { + for ki := 0; ki < format.MaxTags; ki++ { if ki == format.StringTopTagIndexV3 { continue } diff --git a/internal/api/tscache.go b/internal/api/tscache.go index fd5194561..483816eac 100644 --- a/internal/api/tscache.go +++ b/internal/api/tscache.go @@ -35,8 +35,8 @@ type tsSelectRow struct { // all numeric tags are stored as int32 to save space type tsTags struct { - tag [format.NewMaxTags]int64 - stag [format.NewMaxTags]string + tag [format.MaxTags]int64 + stag [format.MaxTags]string shardNum uint32 stagCount int } diff --git a/internal/data_model/bucket.go b/internal/data_model/bucket.go index e403ef492..8cd451314 100644 --- a/internal/data_model/bucket.go +++ b/internal/data_model/bucket.go @@ -14,7 +14,6 @@ import ( "pgregory.net/rand" - "github.com/dchest/siphash" "github.com/hrissan/tdigest" "github.com/zeebo/xxh3" @@ -95,10 +94,6 @@ func (t TagUnionBytes) Empty() bool { return t.I == 0 && len(t.S) == 0 } -// Randomly selected, do not change. Which keys go to which shard depends on this. -const sipKeyA = 0x3605bf49d8e3adf2 -const sipKeyB = 0xc302580679a8cef2 - func (s *ItemCounter) Count() float64 { return s.counter } func (k *Key) SetSTag(i int, s string) { @@ -152,30 +147,6 @@ func AggKey(t uint32, m int32, k [format.MaxTags]int32, hostTagId int32, shardTa return &key } -func (k *Key) Hash() uint64 { - var b [4 + 4*format.MaxTags]byte - // timestamp is not part of shard - binary.LittleEndian.PutUint32(b[:], uint32(k.Metric)) - binary.LittleEndian.PutUint32(b[4+0*4:], uint32(k.Tags[0])) - binary.LittleEndian.PutUint32(b[4+1*4:], uint32(k.Tags[1])) - binary.LittleEndian.PutUint32(b[4+2*4:], uint32(k.Tags[2])) - binary.LittleEndian.PutUint32(b[4+3*4:], uint32(k.Tags[3])) - binary.LittleEndian.PutUint32(b[4+4*4:], uint32(k.Tags[4])) - binary.LittleEndian.PutUint32(b[4+5*4:], uint32(k.Tags[5])) - binary.LittleEndian.PutUint32(b[4+6*4:], uint32(k.Tags[6])) - binary.LittleEndian.PutUint32(b[4+7*4:], uint32(k.Tags[7])) - binary.LittleEndian.PutUint32(b[4+8*4:], uint32(k.Tags[8])) - binary.LittleEndian.PutUint32(b[4+9*4:], uint32(k.Tags[9])) - binary.LittleEndian.PutUint32(b[4+10*4:], uint32(k.Tags[10])) - binary.LittleEndian.PutUint32(b[4+11*4:], uint32(k.Tags[11])) - binary.LittleEndian.PutUint32(b[4+12*4:], uint32(k.Tags[12])) - binary.LittleEndian.PutUint32(b[4+13*4:], uint32(k.Tags[13])) - binary.LittleEndian.PutUint32(b[4+14*4:], uint32(k.Tags[14])) - binary.LittleEndian.PutUint32(b[4+15*4:], uint32(k.Tags[15])) - const _ = uint(16 - format.MaxTags) // compile time assert to manually add new keys above - return siphash.Hash(sipKeyA, sipKeyB, b[:]) -} - // returns possibly reallocated scratch func (k *Key) XXHash(scratch []byte) ([]byte, uint64) { scratch, _ = k.MarshalAppend(scratch[:0]) diff --git a/internal/data_model/known_tags.go b/internal/data_model/known_tags.go index fd51a282f..0bea7072f 100644 --- a/internal/data_model/known_tags.go +++ b/internal/data_model/known_tags.go @@ -62,7 +62,7 @@ func publishDraftTags(meta *format.MetricMetaValue, knownTags []KnownTag) int { // pass } } - if x < 1 || format.NewMaxTags <= x || (x < len(meta.Tags) && meta.Tags[x].Name != "") { + if x < 1 || format.MaxTags <= x || (x < len(meta.Tags) && meta.Tags[x].Name != "") { continue } draftTag.Name = knownTag.Name diff --git a/internal/data_model/mapped_metric_header.go b/internal/data_model/mapped_metric_header.go index 1c8a93fb5..aaa19c816 100644 --- a/internal/data_model/mapped_metric_header.go +++ b/internal/data_model/mapped_metric_header.go @@ -35,7 +35,7 @@ type MappedMetricHeader struct { CheckedTagIndex int // we check tags one by one, remembering position here, between invocations of mapTags ValuesChecked bool // infs, nans, etc. This might be expensive, so done only once - OriginalTagValues [format.NewMaxTags][]byte + OriginalTagValues [format.MaxTags][]byte // original strings values as sent by user. Hash of those is stable between agents independent of // mappings, so used as a resolution hash to deterministically place same rows into same resolution buckets diff --git a/internal/data_model/query_filter.go b/internal/data_model/query_filter.go index d27a8f4b4..dc99d93c3 100644 --- a/internal/data_model/query_filter.go +++ b/internal/data_model/query_filter.go @@ -22,7 +22,7 @@ type QueryFilter struct { type TagFilters struct { Metrics []*format.MetricMetaValue - Tags [format.NewMaxTags]TagFilter + Tags [format.MaxTags]TagFilter } type TagFilter struct { diff --git a/internal/format/builtin.go b/internal/format/builtin.go index 6c70d86fc..614dee30d 100644 --- a/internal/format/builtin.go +++ b/internal/format/builtin.go @@ -217,7 +217,7 @@ func init() { // v.WithAgentEnvRouteArch = false // v.WithAggregatorID = false } - for i := 0; i < NewMaxTags; i++ { + for i := 0; i < MaxTags; i++ { name := strconv.Itoa(i) legacyName := legacyTagIDPrefix + name tagIDsLegacy = append(tagIDsLegacy, legacyName) diff --git a/internal/format/format.go b/internal/format/format.go index 6c32eb8e9..92811da9c 100644 --- a/internal/format/format.go +++ b/internal/format/format.go @@ -25,8 +25,7 @@ import ( ) const ( - MaxTags = 16 - NewMaxTags = 48 + MaxTags = 48 MaxDraftTags = 128 MaxStringLen = 128 // both for normal tags and _s, _h tags (string tops, hostnames) @@ -486,9 +485,9 @@ func (m *MetricMetaValue) RestoreCachedInfo() error { } m.PreKeyIndex = -1 tags := m.Tags - if len(tags) > NewMaxTags { // prevent various overflows in code - tags = tags[:NewMaxTags] - err = multierr.Append(err, fmt.Errorf("too many tags, limit is: %d", NewMaxTags)) + if len(tags) > MaxTags { // prevent various overflows in code + tags = tags[:MaxTags] + err = multierr.Append(err, fmt.Errorf("too many tags, limit is: %d", MaxTags)) } for name, tag := range m.TagsDraft { // in compact journal we clear tag.Name, so we must restore @@ -519,7 +518,7 @@ func (m *MetricMetaValue) RestoreCachedInfo() error { } tag.Raw = tag.RawKind != "" // Raw is serialized for v2 agents only tag.raw64 = IsRaw64Kind(tag.RawKind) - if tag.raw64 && tag.Index >= NewMaxTags-1 { // for now, to avoid overflows in v2 and v3 mapping + if tag.raw64 && tag.Index >= MaxTags-1 { // for now, to avoid overflows in v2 and v3 mapping err = multierr.Append(err, fmt.Errorf("last tag cannot be raw64 kind %q of tag %d", tag.RawKind, i)) tag.raw64 = false } @@ -1262,7 +1261,7 @@ func SameCompactMetric(a, b *MetricMetaValue) bool { a.RoundSampleFactors != b.RoundSampleFactors { return false } - for i := 0; i < NewMaxTags; i++ { + for i := 0; i < MaxTags; i++ { ta := a.Name2Tag(TagID(i)) tb := b.Name2Tag(TagID(i)) if !SameCompactTag(ta, tb) { diff --git a/internal/loadgen/generator.go b/internal/loadgen/generator.go index 0b6ec9dff..77cc49a1e 100644 --- a/internal/loadgen/generator.go +++ b/internal/loadgen/generator.go @@ -32,6 +32,60 @@ type GenericMetric interface { Ensure(ctx context.Context, c *api.Client) } +type manyTagsValueMetric struct { + name string + tags statshouse.Tags + resolution int + + value float64 +} + +func (m *manyTagsValueMetric) Name() string { + return m.name +} + +func (m *manyTagsValueMetric) Write(c *statshouse.Client) { + c.Value(m.name, m.tags, m.value) +} + +func (m *manyTagsValueMetric) Update(now time.Time, rng *rand.Rand) { + ts := now.Unix() + switch m.resolution { + case 1: + m.value = math.Sin(math.Pi * 2. * float64(ts%300) / 300.) + case 15: + m.value = math.Sin(math.Pi * 2. * float64(ts%600) / 600.) + case 60: + m.value = math.Sin(math.Pi * 2. * float64(ts%1800) / 1800.) + default: + panic("unexpected resolution") + } +} + +func (m *manyTagsValueMetric) Ensure(ctx context.Context, c *api.Client) { + metric, err := c.GetMetric(ctx, m.name) + if err != nil { + log.Printf("error getting metric: %v", err) + } + metric.Metric.Tags = []format.MetricMetaTag{ + { + Description: "environment", + }, + } + for i := 1; i < 48; i++ { + metric.Metric.Tags = append(metric.Metric.Tags, format.MetricMetaTag{ + Name: fmt.Sprintf("tag_%d", i), + }) + } + metric.Metric.Name = m.name + metric.Metric.Resolution = m.resolution + metric.Metric.Kind = format.MetricKindValue + err = c.PostMetric(ctx, metric) + if err != nil { + log.Printf("error creating metric: %v", err) + } +} + type valueMetric struct { name string tags statshouse.NamedTags @@ -276,6 +330,19 @@ func (g *Generator) AddConstValue(resolution int, idx int) { g.metrics = append(g.metrics, &m) } +func (g *Generator) AddConstValueManyTags(resolution int, idx int) { + tags := statshouse.Tags{} + for i := range tags { + tags[i] = fmt.Sprint("tag_", i) + } + m := manyTagsValueMetric{ + name: metricPrefixG + "const_val_many_tags_" + fmt.Sprint(resolution), + tags: tags, + resolution: resolution, + } + g.metrics = append(g.metrics, &m) +} + func (g *Generator) AddConstValueHost(resolution int, idx int, host string) { m := valueMetric{ name: metricPrefixG + "const_val_host_" + fmt.Sprint(resolution), diff --git a/internal/loadgen/loadgen.go b/internal/loadgen/loadgen.go index c08b24bd9..ec40ef18a 100644 --- a/internal/loadgen/loadgen.go +++ b/internal/loadgen/loadgen.go @@ -94,6 +94,7 @@ func addMetrics(g *Generator, resolution int, idx int) { g.AddConstPercentile(resolution, idx) g.AddConstValueHost(resolution, idx, "host_1") g.AddConstValueHost(resolution, idx, "host_2") + g.AddConstValueManyTags(resolution, idx) // metrics with changing tag values g.AddChangingCounter(resolution, idx) g.AddChangingValue(resolution, idx) diff --git a/internal/mapping/pipeline_v2.go b/internal/mapping/pipeline_v2.go index 3e1efe335..87e244699 100644 --- a/internal/mapping/pipeline_v2.go +++ b/internal/mapping/pipeline_v2.go @@ -93,7 +93,7 @@ func (mp *mapPipelineV2) mapTags(h *data_model.MappedMetricHeader, metric *tlsta // We could arguably call h.SetKey, but there is very little difference in semantic to care continue } - if tagMeta.Index+1 < format.MaxTags { // TODO - remove after NewMaxTags + if tagMeta.Index+1 < format.MaxTags { // TODO - remove after MaxTags h.SetTag(tagMeta.Index+1, hi, tagIDKey+1) // last tag is never Raw64, checked by RestoreCachedInfo } h.SetTag(tagMeta.Index, lo, tagIDKey) diff --git a/internal/promql/engine.go b/internal/promql/engine.go index 3ffe9f323..b95307c2f 100644 --- a/internal/promql/engine.go +++ b/internal/promql/engine.go @@ -1131,7 +1131,7 @@ func (ev *evaluator) buildSeriesQuery(ctx context.Context, sel *parser.VectorSel } // filtering var ( - emptyCount [format.NewMaxTags]int // number of "MatchEqual" or "MatchRegexp" filters which are guaranteed to yield empty response + emptyCount [format.MaxTags]int // number of "MatchEqual" or "MatchRegexp" filters which are guaranteed to yield empty response ) for _, matcher := range sel.LabelMatchers { if strings.HasPrefix(matcher.Name, "__") { diff --git a/internal/sharding/sharding.go b/internal/sharding/sharding.go index 92512ffbc..e5aba1ce5 100644 --- a/internal/sharding/sharding.go +++ b/internal/sharding/sharding.go @@ -6,7 +6,7 @@ import ( ) // legacyKeyHash will be 0 for all new sharding strategies -func Shard(key *data_model.Key, meta *format.MetricMetaValue, numShards int, shardByMetricCount uint32, newShardingByName string) (shardID uint32, newStrategy bool, weightMul int, legacyKeyHash uint64) { +func Shard(key *data_model.Key, meta *format.MetricMetaValue, numShards int, shardByMetricCount uint32, newShardingByName string, scratch *[]byte) (shardID uint32, newStrategy bool, weightMul int, legacyKeyHash uint64) { s := meta.ShardStrategy if s == "" { if meta.Name <= newShardingByName { @@ -23,7 +23,14 @@ func Shard(key *data_model.Key, meta *format.MetricMetaValue, numShards int, sha shard := uint32(key.Metric) % shardByMetricCount return shard, true, numShards, 0 default: // including format.ShardByTagsHsh - legacyKeyHash = key.Hash() + var scr []byte + if scratch != nil { + scr = *scratch + } + scr, legacyKeyHash = key.XXHash(scr) + if scratch != nil { + *scratch = scr + } shard := shardByMappedTags(legacyKeyHash, numShards) return shard, false, 1, legacyKeyHash } diff --git a/internal/sharding/sharding_test.go b/internal/sharding/sharding_test.go index 12e3c0116..da8388439 100644 --- a/internal/sharding/sharding_test.go +++ b/internal/sharding/sharding_test.go @@ -33,21 +33,21 @@ func TestShard(t *testing.T) { numShards: 16, newShardingByName: "~", newStrategy: false, // meta has priority - }, 8}, + }, 13}, {"ok-by-tags-hash-2", args{ key: metric2, meta: &format.MetricMetaValue{Name: "a", ShardStrategy: format.ShardByTagsHash}, numShards: 16, newShardingByName: "~", newStrategy: false, // meta has priority - }, 15}, + }, 1}, {"ok-by-tags-builtin", args{ key: metricBuiltin, meta: &format.MetricMetaValue{Name: "a", ShardStrategy: format.ShardByTagsHash}, numShards: 16, newShardingByName: "~", newStrategy: false, // meta has priority - }, 5}, + }, 11}, {"ok-fixed-shard", args{ key: metric1, @@ -78,7 +78,7 @@ func TestShard(t *testing.T) { meta: &format.MetricMetaValue{Name: "a"}, numShards: 16, newStrategy: false, - }, 8}, + }, 13}, {"new-sharding-true-user-metric", args{ key: metric1, @@ -108,12 +108,13 @@ func TestShard(t *testing.T) { numShards: 16, newShardingByName: "a", newStrategy: false, - }, 8}, + }, 13}, } + scratch := make([]byte, 0) for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - gotShard, newStrategy, _, gotHash := Shard(&tt.args.key, tt.args.meta, tt.args.numShards, uint32(tt.args.numShards), tt.args.newShardingByName) + gotShard, newStrategy, _, gotHash := Shard(&tt.args.key, tt.args.meta, tt.args.numShards, uint32(tt.args.numShards), tt.args.newShardingByName, &scratch) if gotShard != tt.expectedShard { t.Errorf("Sharding() = %v, want %v", gotShard, tt.expectedShard) } diff --git a/localdebug/run-agent.sh b/localdebug/run-agent.sh index 6650ff113..1ad0e5596 100755 --- a/localdebug/run-agent.sh +++ b/localdebug/run-agent.sh @@ -2,7 +2,8 @@ set -e mkdir -p cache/agent/ -../target/statshouse -agent --hardware-metric-scrape-disable --cluster=statlogs2 --hostname=agent1 --agg-addr=127.0.0.1:13336,127.0.0.1:13346,127.0.0.1:13356 --cache-dir=cache/agent "$@" +../target/statshouse -agent --cluster=statlogs2 --hostname=agent1 --agg-addr=127.0.0.1:13336,127.0.0.1:13346,127.0.0.1:13356 --cache-dir=cache/agent "$@" +# ../target/statshouse -agent --hardware-metric-scrape-disable --cluster=statlogs2 --hostname=agent1 --agg-addr=127.0.0.1:13336,127.0.0.1:13346,127.0.0.1:13356 --cache-dir=cache/agent "$@" # other useful options you sometimes want to add # --cores-udp=2 --listen-addr-ipv6=[::1]:13337 --listen-addr-unix=@statshouse