Skip to content

Commit

Permalink
spanconfig/store: intern span configs
Browse files Browse the repository at this point in the history
- Avoid the expensive proto.Equal() when computing split keys;
- Reduce memory overhead of the data structure

    $ dev bench pkg/spanconfig/spanconfigstore -f=BenchmarkStoreComputeSplitKey -v

    BenchmarkStoreComputeSplitKey
    BenchmarkStoreComputeSplitKey/num-entries=10000
    BenchmarkStoreComputeSplitKey/num-entries=10000-10                 90323 ns/op
    BenchmarkStoreComputeSplitKey/num-entries=100000
    BenchmarkStoreComputeSplitKey/num-entries=100000-10               915936 ns/op
    BenchmarkStoreComputeSplitKey/num-entries=1000000
    BenchmarkStoreComputeSplitKey/num-entries=1000000-10             9575781 ns/op

    $ benchstat old.txt new.txt # from previous commit

    name                                         old time/op  new time/op  delta
    StoreComputeSplitKey/num-entries=10000-10     431µs ± 0%    90µs ± 0%   ~     (p=1.000 n=1+1)
    StoreComputeSplitKey/num-entries=100000-10   4.31ms ± 0%  0.92ms ± 0%   ~     (p=1.000 n=1+1)
    StoreComputeSplitKey/num-entries=1000000-10  43.8ms ± 0%   9.6ms ± 0%   ~     (p=1.000 n=1+1)

Release note: None
  • Loading branch information
irfansharif committed Apr 27, 2022
1 parent 3bf4a48 commit 3b0df85
Show file tree
Hide file tree
Showing 7 changed files with 241 additions and 100 deletions.
2 changes: 2 additions & 0 deletions pkg/spanconfig/spanconfigstore/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "spanconfigstore",
srcs = [
"interner.go",
"span_store.go",
"store.go",
"system_store.go",
Expand All @@ -19,6 +20,7 @@ go_library(
"//pkg/spanconfig",
"//pkg/util/iterutil",
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil", # keep
"@com_github_cockroachdb_errors//:errors",
Expand Down
88 changes: 88 additions & 0 deletions pkg/spanconfig/spanconfigstore/interner.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package spanconfigstore

import (
"context"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
)

type internerID uint64

// interner interns span config protos. It's a ref-counted data structure that
// maintains a single copy of every unique config that that's been added to
// it. Configs that are maintained are identified/retrivable using an
// internerID. Configs can also be removed, and if there are no more references
// to it, we no longer hold onto it in memory.
type interner struct {
internIDAlloc internerID

configToID map[string]internerID
idToConfig map[internerID]roachpb.SpanConfig
idToRefCount map[internerID]uint64
}

func newInterner() *interner {
return &interner{
configToID: make(map[string]internerID),
idToConfig: make(map[internerID]roachpb.SpanConfig),
idToRefCount: make(map[internerID]uint64),
}
}

func (i *interner) add(ctx context.Context, conf roachpb.SpanConfig) internerID {
marshalled, err := protoutil.Marshal(&conf)
if err != nil {
log.Fatalf(ctx, "%v", err)
}

if id, found := i.configToID[string(marshalled)]; found {
i.idToRefCount[id]++
return id
}

i.internIDAlloc++

id := i.internIDAlloc
i.configToID[string(marshalled)] = id
i.idToConfig[i.internIDAlloc] = conf
i.idToRefCount[id] = 1
return id
}

func (i *interner) get(id internerID) (roachpb.SpanConfig, bool) {
conf, found := i.idToConfig[id]
return conf, found
}

func (i *interner) remove(ctx context.Context, id internerID) {
conf, found := i.idToConfig[id]
if !found {
return // nothing to do
}

i.idToRefCount[id]--
if i.idToRefCount[id] != 0 {
return // nothing to do
}

marshalled, err := protoutil.Marshal(&conf)
if err != nil {
log.Infof(ctx, "%v", err)
}

delete(i.idToConfig, id)
delete(i.idToRefCount, id)
delete(i.configToID, string(marshalled))
}
Loading

0 comments on commit 3b0df85

Please sign in to comment.