Skip to content

Commit

Permalink
storage: re-add MVCC incremental iterator tests
Browse files Browse the repository at this point in the history
These tests exercise the Go MVCCIncrementalIterator which is no longer
used directly in CockroachDB. However, the MVCCIncrementalIterator is
used as an oracle to test the C++ implementation. Therefore, it would
still be beneficial to have these tests present as long as we plan to
keep these 2 implementations in sync.

Release note: None
  • Loading branch information
pbardea committed Feb 26, 2020
1 parent c5aa1a8 commit a455afa
Show file tree
Hide file tree
Showing 2 changed files with 518 additions and 0 deletions.
92 changes: 92 additions & 0 deletions pkg/storage/engine/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -53,6 +55,96 @@ type benchDataOptions struct {
transactional bool
}

// loadTestData writes numKeys keys in numBatches separate batches. Keys are
// written in order. Every key in a given batch has the same MVCC timestamp;
// batch timestamps start at batchTimeSpan and increase in intervals of
// batchTimeSpan.
//
// Importantly, writing keys in order convinces RocksDB to output one SST per
// batch, where each SST contains keys of only one timestamp. E.g., writing A,B
// at t0 and C at t1 will create two SSTs: one for A,B that only contains keys
// at t0, and one for C that only contains keys at t1. Conversely, writing A, C
// at t0 and B at t1 would create just one SST that contained A,B,C (due to an
// immediate compaction).
//
// The creation of the database is time consuming, so the caller can choose
// whether to use a temporary or permanent location.
func loadTestData(dir string, numKeys, numBatches, batchTimeSpan, valueBytes int) (Engine, error) {
ctx := context.Background()

exists := true
if _, err := os.Stat(dir); os.IsNotExist(err) {
exists = false
}

eng, err := NewRocksDB(
RocksDBConfig{
StorageConfig: base.StorageConfig{
Settings: cluster.MakeTestingClusterSettings(),
Dir: dir,
},
},
RocksDBCache{},
)
if err != nil {
return nil, err
}

if exists {
testutils.ReadAllFiles(filepath.Join(dir, "*"))
return eng, nil
}

log.Infof(context.Background(), "creating test data: %s", dir)

// Generate the same data every time.
rng := rand.New(rand.NewSource(1449168817))

keys := make([]roachpb.Key, numKeys)
for i := 0; i < numKeys; i++ {
keys[i] = roachpb.Key(encoding.EncodeUvarintAscending([]byte("key-"), uint64(i)))
}

sstTimestamps := make([]int64, numBatches)
for i := 0; i < len(sstTimestamps); i++ {
sstTimestamps[i] = int64((i + 1) * batchTimeSpan)
}

var batch Batch
var minWallTime int64
for i, key := range keys {
if scaled := len(keys) / numBatches; (i % scaled) == 0 {
if i > 0 {
log.Infof(ctx, "committing (%d/~%d)", i/scaled, numBatches)
if err := batch.Commit(false /* sync */); err != nil {
return nil, err
}
batch.Close()
if err := eng.Flush(); err != nil {
return nil, err
}
}
batch = eng.NewBatch()
minWallTime = sstTimestamps[i/scaled]
}
timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))}
value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes))
value.InitChecksum(key)
if err := MVCCPut(ctx, batch, nil, key, timestamp, value, nil); err != nil {
return nil, err
}
}
if err := batch.Commit(false /* sync */); err != nil {
return nil, err
}
batch.Close()
if err := eng.Flush(); err != nil {
return nil, err
}

return eng, nil
}

// setupMVCCData writes up to numVersions values at each of numKeys
// keys. The number of versions written for each key is chosen
// randomly according to a uniform distribution. Each successive
Expand Down
Loading

0 comments on commit a455afa

Please sign in to comment.