github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/storage/disk_map_test.go (about) 1 // Copyright 2017 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package storage 12 13 import ( 14 "bytes" 15 "context" 16 "fmt" 17 "io/ioutil" 18 "math/rand" 19 "os" 20 "sort" 21 "strings" 22 "testing" 23 24 "github.com/cockroachdb/cockroach/pkg/base" 25 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" 26 "github.com/cockroachdb/cockroach/pkg/roachpb" 27 "github.com/cockroachdb/cockroach/pkg/testutils" 28 "github.com/cockroachdb/cockroach/pkg/util/encoding" 29 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 30 "github.com/cockroachdb/cockroach/pkg/util/timeutil" 31 "github.com/cockroachdb/datadriven" 32 "github.com/cockroachdb/pebble" 33 ) 34 35 // put gives a quick interface for writing a KV pair to a rocksDBMap in tests. 36 func (r *rocksDBMap) put(k []byte, v []byte) error { 37 return r.store.Put(r.makeKeyWithTimestamp(k), v) 38 } 39 40 // Helper function to run a datadriven test for a provided diskMap 41 func runTestForEngine(ctx context.Context, t *testing.T, filename string, engine diskmap.Factory) { 42 diskMaps := make(map[string]diskmap.SortedDiskMap) 43 44 datadriven.RunTest(t, filename, func(t *testing.T, d *datadriven.TestData) string { 45 if d.Cmd == "raw-count" { 46 var keyCount int 47 // Trying to build a common interface to RocksDB and Pebble's iterator 48 // creation and usage code involves too much test-specific code, so use 49 // a type switch with implementation-specific code instead. 50 switch e := engine.(type) { 51 case *rocksDBTempEngine: 52 iter := e.db.NewIterator(IterOptions{UpperBound: roachpb.KeyMax}) 53 defer iter.Close() 54 55 for iter.SeekGE(NilKey); ; iter.Next() { 56 valid, err := iter.Valid() 57 if valid && err == nil { 58 keyCount++ 59 } else if err != nil { 60 return fmt.Sprintf("err=%v\n", err) 61 } else { 62 break 63 } 64 } 65 case *pebbleTempEngine: 66 iter := e.db.NewIter(&pebble.IterOptions{UpperBound: roachpb.KeyMax}) 67 68 defer func() { 69 if err := iter.Close(); err != nil { 70 t.Fatal(err) 71 } 72 }() 73 74 for valid := iter.First(); valid; valid = iter.Next() { 75 keyCount++ 76 } 77 default: 78 return "unsupported engine type" 79 } 80 return fmt.Sprintf("count=%d\n", keyCount) 81 } 82 83 // All other commands require a map name 84 if len(d.CmdArgs) < 1 { 85 return "no map name specified" 86 } 87 88 switch d.Cmd { 89 case "new-map": 90 duplicates := false 91 for _, arg := range d.CmdArgs[1:] { 92 if arg.Key == "duplicates" { 93 if len(arg.Vals) < 1 || (arg.Vals[0] != "true" && arg.Vals[0] != "false") { 94 return "usage: new-map <name> [duplicates={true, false}]" 95 } else if arg.Vals[0] == "true" { 96 duplicates = true 97 } 98 } 99 } 100 if duplicates { 101 diskMaps[d.CmdArgs[0].Key] = engine.NewSortedDiskMultiMap() 102 } else { 103 diskMaps[d.CmdArgs[0].Key] = engine.NewSortedDiskMap() 104 } 105 106 return "" 107 } 108 109 diskMap, ok := diskMaps[d.CmdArgs[0].Key] 110 111 if !ok { 112 return fmt.Sprintf("unknown map: %s", d.CmdArgs[0].Key) 113 } 114 115 switch d.Cmd { 116 case "close-map": 117 diskMap.Close(ctx) 118 return "" 119 120 case "batch": 121 batch := diskMap.NewBatchWriter() 122 defer func() { 123 if err := batch.Close(ctx); err != nil { 124 t.Fatal(err) 125 } 126 }() 127 128 for _, line := range strings.Split(d.Input, "\n") { 129 parts := strings.Fields(line) 130 if len(parts) == 0 { 131 continue 132 } 133 switch parts[0] { 134 case "put": 135 if len(parts) != 3 { 136 return fmt.Sprintf("put <key> <value>\n") 137 } 138 err := batch.Put([]byte(strings.TrimSpace(parts[1])), []byte(strings.TrimSpace(parts[2]))) 139 if err != nil { 140 return err.Error() 141 } 142 default: 143 return fmt.Sprintf("unknown op: %s", parts[0]) 144 } 145 } 146 return "" 147 148 case "iter": 149 iter := diskMap.NewIterator() 150 defer iter.Close() 151 var b bytes.Buffer 152 for _, line := range strings.Split(d.Input, "\n") { 153 parts := strings.Fields(line) 154 if len(parts) == 0 { 155 continue 156 } 157 switch parts[0] { 158 case "next": 159 iter.Next() 160 161 case "rewind": 162 iter.Rewind() 163 164 case "seek": 165 if len(parts) != 2 { 166 return fmt.Sprintf("seek <key>\n") 167 } 168 iter.SeekGE([]byte(strings.TrimSpace(parts[1]))) 169 default: 170 return fmt.Sprintf("unknown op: %s", parts[0]) 171 } 172 valid, err := iter.Valid() 173 if valid && err == nil { 174 fmt.Fprintf(&b, "%s:%s\n", iter.UnsafeKey(), iter.UnsafeValue()) 175 } else if err != nil { 176 fmt.Fprintf(&b, "err=%v\n", err) 177 } else { 178 fmt.Fprintf(&b, ".\n") 179 } 180 } 181 return b.String() 182 default: 183 // No other commands supported. 184 return fmt.Sprintf("unsupported command: %s", d.Cmd) 185 } 186 }) 187 } 188 189 func TestRocksDBMap(t *testing.T) { 190 defer leaktest.AfterTest(t)() 191 ctx := context.Background() 192 e := newRocksDBInMem(roachpb.Attributes{}, 1<<20) 193 defer e.Close() 194 195 runTestForEngine(ctx, t, "testdata/diskmap", &rocksDBTempEngine{db: e}) 196 } 197 198 func TestRocksDBMultiMap(t *testing.T) { 199 defer leaktest.AfterTest(t)() 200 ctx := context.Background() 201 e := newRocksDBInMem(roachpb.Attributes{}, 1<<20) 202 defer e.Close() 203 204 runTestForEngine(ctx, t, "testdata/diskmap_duplicates", &rocksDBTempEngine{db: e}) 205 } 206 207 func TestRocksDBMapClose(t *testing.T) { 208 defer leaktest.AfterTest(t)() 209 ctx := context.Background() 210 e := newRocksDBInMem(roachpb.Attributes{}, 1<<20) 211 defer e.Close() 212 213 decodeKey := func(v []byte) []byte { 214 var err error 215 v, _, err = encoding.DecodeUvarintAscending(v) 216 if err != nil { 217 t.Fatal(err) 218 } 219 return v 220 } 221 222 getSSTables := func() string { 223 ssts := e.GetSSTables() 224 sort.Slice(ssts, func(i, j int) bool { 225 a, b := ssts[i], ssts[j] 226 if a.Level < b.Level { 227 return true 228 } 229 if a.Level > b.Level { 230 return false 231 } 232 return a.Start.Less(b.Start) 233 }) 234 var buf bytes.Buffer 235 fmt.Fprintf(&buf, "\n") 236 for i := range ssts { 237 fmt.Fprintf(&buf, "%d: %s - %s\n", 238 ssts[i].Level, decodeKey(ssts[i].Start.Key), decodeKey(ssts[i].End.Key)) 239 } 240 return buf.String() 241 } 242 243 verifySSTables := func(expected string) { 244 actual := getSSTables() 245 if expected != actual { 246 t.Fatalf("expected%sgot%s", expected, actual) 247 } 248 if testing.Verbose() { 249 fmt.Printf("%s", actual) 250 } 251 } 252 253 diskMap := newRocksDBMap(e, false /* allowDuplicates */) 254 255 // Put a small amount of data into the disk map. 256 const letters = "abcdefghijklmnopqrstuvwxyz" 257 for i := range letters { 258 k := []byte{letters[i]} 259 if err := diskMap.put(k, k); err != nil { 260 t.Fatal(err) 261 } 262 } 263 264 // Force the data to disk. 265 if err := e.Flush(); err != nil { 266 t.Fatal(err) 267 } 268 269 // Force it to a lower-level. This is done so as to avoid the automatic 270 // compactions out of L0 that would normally occur. 271 if err := e.Compact(); err != nil { 272 t.Fatal(err) 273 } 274 275 // Verify we have a single sstable. 276 verifySSTables(` 277 6: a - z 278 `) 279 280 // Close the disk map. This should both delete the data, and initiate 281 // compactions for the deleted data. 282 diskMap.Close(ctx) 283 284 // Wait for the data stored in the engine to disappear. 285 testutils.SucceedsSoon(t, func() error { 286 actual := getSSTables() 287 if testing.Verbose() { 288 fmt.Printf("%s", actual) 289 } 290 if actual != "\n" { 291 return fmt.Errorf("%s", actual) 292 } 293 return nil 294 }) 295 } 296 297 func BenchmarkRocksDBMapWrite(b *testing.B) { 298 dir, err := ioutil.TempDir("", "BenchmarkRocksDBMapWrite") 299 if err != nil { 300 b.Fatal(err) 301 } 302 defer func() { 303 if err := os.RemoveAll(dir); err != nil { 304 b.Fatal(err) 305 } 306 }() 307 ctx := context.Background() 308 tempEngine, _, err := NewRocksDBTempEngine(base.TempStorageConfig{Path: dir}, base.DefaultTestStoreSpec) 309 if err != nil { 310 b.Fatal(err) 311 } 312 defer tempEngine.Close() 313 314 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 315 316 for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { 317 b.Run(fmt.Sprintf("InputSize%d", inputSize), func(b *testing.B) { 318 for i := 0; i < b.N; i++ { 319 func() { 320 diskMap := tempEngine.NewSortedDiskMap() 321 defer diskMap.Close(ctx) 322 batchWriter := diskMap.NewBatchWriter() 323 // This Close() flushes writes. 324 defer func() { 325 if err := batchWriter.Close(ctx); err != nil { 326 b.Fatal(err) 327 } 328 }() 329 for j := 0; j < inputSize; j++ { 330 k := fmt.Sprintf("%d", rng.Int()) 331 v := fmt.Sprintf("%d", rng.Int()) 332 if err := batchWriter.Put([]byte(k), []byte(v)); err != nil { 333 b.Fatal(err) 334 } 335 } 336 }() 337 } 338 }) 339 } 340 } 341 342 func BenchmarkRocksDBMapIteration(b *testing.B) { 343 if testing.Short() { 344 b.Skip("short flag") 345 } 346 dir, err := ioutil.TempDir("", "BenchmarkRocksDBMapIteration") 347 if err != nil { 348 b.Fatal(err) 349 } 350 defer func() { 351 if err := os.RemoveAll(dir); err != nil { 352 b.Fatal(err) 353 } 354 }() 355 tempEngine, _, err := NewRocksDBTempEngine(base.TempStorageConfig{Path: dir}, base.DefaultTestStoreSpec) 356 if err != nil { 357 b.Fatal(err) 358 } 359 defer tempEngine.Close() 360 361 diskMap := tempEngine.NewSortedDiskMap().(*rocksDBMap) 362 defer diskMap.Close(context.Background()) 363 364 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 365 366 for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { 367 for i := 0; i < inputSize; i++ { 368 k := fmt.Sprintf("%d", rng.Int()) 369 v := fmt.Sprintf("%d", rng.Int()) 370 if err := diskMap.put([]byte(k), []byte(v)); err != nil { 371 b.Fatal(err) 372 } 373 } 374 375 b.Run(fmt.Sprintf("InputSize%d", inputSize), func(b *testing.B) { 376 for j := 0; j < b.N; j++ { 377 i := diskMap.NewIterator() 378 for i.Rewind(); ; i.Next() { 379 if ok, err := i.Valid(); err != nil { 380 b.Fatal(err) 381 } else if !ok { 382 break 383 } 384 i.UnsafeKey() 385 i.UnsafeValue() 386 } 387 i.Close() 388 } 389 }) 390 } 391 } 392 393 func TestPebbleMap(t *testing.T) { 394 defer leaktest.AfterTest(t)() 395 ctx := context.Background() 396 dir, cleanup := testutils.TempDir(t) 397 defer cleanup() 398 399 e, _, err := NewPebbleTempEngine(ctx, base.TempStorageConfig{Path: dir}, base.StoreSpec{}) 400 if err != nil { 401 t.Fatal(err) 402 } 403 defer e.Close() 404 405 runTestForEngine(ctx, t, "testdata/diskmap", e) 406 407 } 408 409 func TestPebbleMultiMap(t *testing.T) { 410 defer leaktest.AfterTest(t)() 411 ctx := context.Background() 412 dir, cleanup := testutils.TempDir(t) 413 defer cleanup() 414 415 e, _, err := NewPebbleTempEngine(ctx, base.TempStorageConfig{Path: dir}, base.StoreSpec{}) 416 if err != nil { 417 t.Fatal(err) 418 } 419 defer e.Close() 420 421 runTestForEngine(ctx, t, "testdata/diskmap_duplicates_pebble", e) 422 423 } 424 425 func TestPebbleMapClose(t *testing.T) { 426 defer leaktest.AfterTest(t)() 427 ctx := context.Background() 428 dir, cleanup := testutils.TempDir(t) 429 defer cleanup() 430 e, _, err := newPebbleTempEngine(ctx, base.TempStorageConfig{Path: dir}, base.StoreSpec{}) 431 if err != nil { 432 t.Fatal(err) 433 } 434 defer e.Close() 435 436 decodeKey := func(v []byte) []byte { 437 var err error 438 v, _, err = encoding.DecodeUvarintAscending(v) 439 if err != nil { 440 t.Fatal(err) 441 } 442 return v 443 } 444 445 getSSTables := func() string { 446 var buf bytes.Buffer 447 fmt.Fprintf(&buf, "\n") 448 for l, ssts := range e.db.SSTables() { 449 for _, sst := range ssts { 450 fmt.Fprintf(&buf, "%d: %s - %s\n", 451 l, decodeKey(sst.Smallest.UserKey), decodeKey(sst.Largest.UserKey)) 452 } 453 } 454 return buf.String() 455 } 456 457 verifySSTables := func(expected string) { 458 actual := getSSTables() 459 if expected != actual { 460 t.Fatalf("expected%sgot%s", expected, actual) 461 } 462 if testing.Verbose() { 463 fmt.Printf("%s", actual) 464 } 465 } 466 467 diskMap := newPebbleMap(e.db, false /* allowDuplicates */) 468 469 // Put a small amount of data into the disk map. 470 bw := diskMap.NewBatchWriter() 471 const letters = "abcdefghijklmnopqrstuvwxyz" 472 for i := range letters { 473 k := []byte{letters[i]} 474 if err := bw.Put(k, k); err != nil { 475 t.Fatal(err) 476 } 477 } 478 if err := bw.Close(ctx); err != nil { 479 t.Fatal(err) 480 } 481 482 // Force the data to disk. 483 if err := e.db.Flush(); err != nil { 484 t.Fatal(err) 485 } 486 487 // Force it to a lower-level. This is done so as to avoid the automatic 488 // compactions out of L0 that would normally occur. 489 if err := e.db.Compact(diskMap.makeKey([]byte{'a'}), diskMap.makeKey([]byte{'z'})); err != nil { 490 t.Fatal(err) 491 } 492 493 // Verify we have a single sstable. 494 verifySSTables(` 495 6: a - z 496 `) 497 498 // Close the disk map. This should both delete the data, and initiate 499 // compactions for the deleted data. 500 diskMap.Close(ctx) 501 502 // Wait for the data stored in the engine to disappear. 503 testutils.SucceedsSoon(t, func() error { 504 actual := getSSTables() 505 if testing.Verbose() { 506 fmt.Printf("%s", actual) 507 } 508 if actual != "\n" { 509 return fmt.Errorf("%s", actual) 510 } 511 return nil 512 }) 513 } 514 515 func BenchmarkPebbleMapWrite(b *testing.B) { 516 dir, err := ioutil.TempDir("", "BenchmarkPebbleMapWrite") 517 if err != nil { 518 b.Fatal(err) 519 } 520 defer func() { 521 if err := os.RemoveAll(dir); err != nil { 522 b.Fatal(err) 523 } 524 }() 525 ctx := context.Background() 526 tempEngine, _, err := NewPebbleTempEngine(ctx, base.TempStorageConfig{Path: dir}, base.DefaultTestStoreSpec) 527 if err != nil { 528 b.Fatal(err) 529 } 530 defer tempEngine.Close() 531 532 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 533 534 for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { 535 b.Run(fmt.Sprintf("InputSize%d", inputSize), func(b *testing.B) { 536 for i := 0; i < b.N; i++ { 537 func() { 538 diskMap := tempEngine.NewSortedDiskMap() 539 defer diskMap.Close(ctx) 540 batchWriter := diskMap.NewBatchWriter() 541 // This Close() flushes writes. 542 defer func() { 543 if err := batchWriter.Close(ctx); err != nil { 544 b.Fatal(err) 545 } 546 }() 547 for j := 0; j < inputSize; j++ { 548 k := fmt.Sprintf("%d", rng.Int()) 549 v := fmt.Sprintf("%d", rng.Int()) 550 if err := batchWriter.Put([]byte(k), []byte(v)); err != nil { 551 b.Fatal(err) 552 } 553 } 554 }() 555 } 556 }) 557 } 558 } 559 560 func BenchmarkPebbleMapIteration(b *testing.B) { 561 if testing.Short() { 562 b.Skip("short flag") 563 } 564 dir, err := ioutil.TempDir("", "BenchmarkPebbleMapIteration") 565 if err != nil { 566 b.Fatal(err) 567 } 568 defer func() { 569 if err := os.RemoveAll(dir); err != nil { 570 b.Fatal(err) 571 } 572 }() 573 ctx := context.Background() 574 tempEngine, _, err := NewPebbleTempEngine(ctx, base.TempStorageConfig{Path: dir}, base.DefaultTestStoreSpec) 575 if err != nil { 576 b.Fatal(err) 577 } 578 defer tempEngine.Close() 579 580 diskMap := tempEngine.NewSortedDiskMap() 581 defer diskMap.Close(ctx) 582 583 rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) 584 585 for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { 586 batchWriter := diskMap.NewBatchWriter() 587 defer func() { 588 if err := batchWriter.Close(ctx); err != nil { 589 b.Fatal(err) 590 } 591 }() 592 593 for i := 0; i < inputSize; i++ { 594 k := fmt.Sprintf("%d", rng.Int()) 595 v := fmt.Sprintf("%d", rng.Int()) 596 if err := batchWriter.Put([]byte(k), []byte(v)); err != nil { 597 b.Fatal(err) 598 } 599 } 600 601 if err := batchWriter.Flush(); err != nil { 602 b.Fatal(err) 603 } 604 605 b.Run(fmt.Sprintf("InputSize%d", inputSize), func(b *testing.B) { 606 for j := 0; j < b.N; j++ { 607 i := diskMap.NewIterator() 608 for i.Rewind(); ; i.Next() { 609 if ok, err := i.Valid(); err != nil { 610 b.Fatal(err) 611 } else if !ok { 612 break 613 } 614 i.UnsafeKey() 615 i.UnsafeValue() 616 } 617 i.Close() 618 } 619 }) 620 } 621 }