Skip to content

Commit

Permalink
*: Stop using RocksDB in Go code
Browse files Browse the repository at this point in the history
This change deletes all RocksDB specific Go code, and updates
all tests that called RocksDB-specific methods to call their Pebble
versions instead.

This is a prerequisite for deleting RocksDB from the codebase.

Release note (general change): RocksDB can no longer be used as
the storage engine. Passing in --storage-engine=rocksdb will
return an error.
  • Loading branch information
itsbilal committed Oct 21, 2020
1 parent 5ddbc35 commit 4970b3d
Show file tree
Hide file tree
Showing 77 changed files with 439 additions and 9,075 deletions.
13 changes: 8 additions & 5 deletions pkg/ccl/importccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload"
"github.com/cockroachdb/cockroach/pkg/workload/tpcc"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -90,16 +91,18 @@ func benchmarkWriteAndLink(b *testing.B, dir string, tables []tableSSTable) {
b.SetBytes(bytes)

ctx := context.Background()
cache := storage.NewRocksDBCache(server.DefaultCacheSize)
defer cache.Release()
cache := pebble.NewCache(server.DefaultCacheSize)
defer cache.Unref()

b.ResetTimer()
for i := 0; i < b.N; i++ {
b.StopTimer()
cfg := storage.RocksDBConfig{
cfg := storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Dir: filepath.Join(dir, `rocksdb`, timeutil.Now().String())}}
db, err := storage.NewRocksDB(cfg, cache)
Dir: filepath.Join(dir, `pebble`, timeutil.Now().String())}}
cfg.Opts = storage.DefaultPebbleOptions()
cfg.Opts.Cache = cache
db, err := storage.NewPebble(context.Background(), cfg)
if err != nil {
b.Fatal(err)
}
Expand Down
9 changes: 6 additions & 3 deletions pkg/ccl/storageccl/engineccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,14 +51,14 @@ func loadTestData(
exists = false
}

eng, err := storage.NewRocksDB(
storage.RocksDBConfig{
eng, err := storage.NewPebble(
ctx,
storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Settings: cluster.MakeTestingClusterSettings(),
Dir: dir,
},
},
storage.RocksDBCache{},
)
if err != nil {
return nil, err
Expand Down Expand Up @@ -147,6 +147,9 @@ func runIterate(
n := 0
startTime := hlc.MinTimestamp
endTime := hlc.Timestamp{WallTime: int64(loadFactor * numBatches * batchTimeSpan)}
if endTime.IsEmpty() {
endTime = endTime.Next()
}
it := makeIterator(eng, startTime, endTime)
defer it.Close()
for it.SeekGE(storage.MVCCKey{}); ; it.Next() {
Expand Down
30 changes: 0 additions & 30 deletions pkg/ccl/storageccl/engineccl/rocksdb.go

This file was deleted.

17 changes: 0 additions & 17 deletions pkg/ccl/storageccl/engineccl/rocksdb_jemalloc.go

This file was deleted.

75 changes: 45 additions & 30 deletions pkg/ccl/storageccl/export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,14 +72,13 @@ func TestExportCmd(t *testing.T) {

var paths []string
var kvs []storage.MVCCKeyValue
ingestFunc := func(kv storage.MVCCKeyValue) error {
kvs = append(kvs, kv)
return nil
}
for _, file := range res.(*roachpb.ExportResponse).Files {
paths = append(paths, file.Path)

sst := storage.MakeRocksDBSstFileReader()
sst, err := storage.NewMemSSTIterator(file.SST, false)
if err != nil {
t.Fatalf("%+v", err)
}
defer sst.Close()

fileContents, err := ioutil.ReadFile(filepath.Join(dir, "foo", file.Path))
Expand All @@ -89,11 +88,20 @@ func TestExportCmd(t *testing.T) {
if !bytes.Equal(fileContents, file.SST) {
t.Fatal("Returned SST and exported SST don't match!")
}
if err := sst.IngestExternalFile(file.SST); err != nil {
t.Fatalf("%+v", err)
}
if err := sst.Iterate(keys.MinKey, keys.MaxKey, ingestFunc); err != nil {
t.Fatalf("%+v", err)
sst.SeekGE(storage.MVCCKey{Key: keys.MinKey})
for {
if valid, err := sst.Valid(); !valid || err != nil {
if err != nil {
t.Fatalf("%+v", err)
}
break
}
newKv := storage.MVCCKeyValue{}
newKv.Key.Key = append(newKv.Key.Key, sst.UnsafeKey().Key...)
newKv.Key.Timestamp = sst.UnsafeKey().Timestamp
newKv.Value = append(newKv.Value, sst.UnsafeValue()...)
kvs = append(kvs, newKv)
sst.Next()
}
}

Expand Down Expand Up @@ -287,10 +295,8 @@ func exportUsingGoIterator(
enableTimeBoundIteratorOptimization bool,
reader storage.Reader,
) ([]byte, error) {
sst, err := storage.MakeRocksDBSstFileWriter()
if err != nil {
return nil, nil //nolint:returnerrcheck
}
memFile := &storage.MemFile{}
sst := storage.MakeIngestionSSTWriter(memFile)
defer sst.Close()

var skipTombstones bool
Expand Down Expand Up @@ -340,18 +346,16 @@ func exportUsingGoIterator(
return nil, err
}
}
if err := sst.Finish(); err != nil {
return nil, err
}

if sst.DataSize() == 0 {
if len(memFile.Data()) == 0 {
// Let the defer Close the sstable.
return nil, nil
}

sstContents, err := sst.Finish()
if err != nil {
return nil, err
}

return sstContents, nil
return memFile.Data(), nil
}

func loadSST(t *testing.T, data []byte, start, end roachpb.Key) []storage.MVCCKeyValue {
Expand All @@ -360,19 +364,30 @@ func loadSST(t *testing.T, data []byte, start, end roachpb.Key) []storage.MVCCKe
return nil
}

sst := storage.MakeRocksDBSstFileReader()
defer sst.Close()

if err := sst.IngestExternalFile(data); err != nil {
sst, err := storage.NewMemSSTIterator(data, false)
if err != nil {
t.Fatal(err)
}
defer sst.Close()

var kvs []storage.MVCCKeyValue
if err := sst.Iterate(start, end, func(kv storage.MVCCKeyValue) error {
kvs = append(kvs, kv)
return nil
}); err != nil {
t.Fatal(err)
sst.SeekGE(storage.MVCCKey{Key: start})
for {
if valid, err := sst.Valid(); !valid || err != nil {
if err != nil {
t.Fatal(err)
}
break
}
if !sst.UnsafeKey().Less(storage.MVCCKey{Key: end}) {
break
}
newKv := storage.MVCCKeyValue{}
newKv.Key.Key = append(newKv.Key.Key, sst.UnsafeKey().Key...)
newKv.Key.Timestamp = sst.UnsafeKey().Timestamp
newKv.Value = append(newKv.Value, sst.UnsafeValue()...)
kvs = append(kvs, newKv)
sst.Next()
}

return kvs
Expand Down
46 changes: 27 additions & 19 deletions pkg/ccl/storageccl/import_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
)

func TestMaxImportBatchSize(t *testing.T) {
Expand Down Expand Up @@ -73,31 +72,40 @@ func slurpSSTablesLatestKey(
defer batch.Close()

for _, path := range paths {
sst := storage.MakeRocksDBSstFileReader()
defer sst.Close()

fileContents, err := ioutil.ReadFile(filepath.Join(dir, path))
sst, err := storage.NewSSTIterator(filepath.Join(dir, path))
if err != nil {
t.Fatalf("%+v", err)
}
if err := sst.IngestExternalFile(fileContents); err != nil {
t.Fatalf("%+v", err)
t.Fatal(err)
}
if err := sst.Iterate(start.Key, end.Key, func(kv storage.MVCCKeyValue) error {
defer sst.Close()

sst.SeekGE(start)
for {
if valid, err := sst.Valid(); !valid || err != nil {
if err != nil {
t.Fatal(err)
}
break
}
if !sst.UnsafeKey().Less(end) {
break
}
var ok bool
kv.Key.Key, ok = kr.rewriteKey(kv.Key.Key)
var newKv storage.MVCCKeyValue
key := sst.UnsafeKey()
newKv.Value = append(newKv.Value, sst.UnsafeValue()...)
newKv.Key.Key = append(newKv.Key.Key, key.Key...)
newKv.Key.Timestamp = key.Timestamp
newKv.Key.Key, ok = kr.rewriteKey(newKv.Key.Key)
if !ok {
return errors.Errorf("could not rewrite key: %s", kv.Key.Key)
t.Fatalf("could not rewrite key: %s", newKv.Key.Key)
}
v := roachpb.Value{RawBytes: kv.Value}
v := roachpb.Value{RawBytes: newKv.Value}
v.ClearChecksum()
v.InitChecksum(kv.Key.Key)
if err := batch.Put(kv.Key, v.RawBytes); err != nil {
return err
v.InitChecksum(newKv.Key.Key)
if err := batch.Put(newKv.Key, v.RawBytes); err != nil {
t.Fatal(err)
}
return nil
}); err != nil {
t.Fatalf("%+v", err)
sst.Next()
}
}

Expand Down
Loading

0 comments on commit 4970b3d

Please sign in to comment.