diff --git a/DEPS.bzl b/DEPS.bzl index ecd4952893e31..1e0c6c11a8a32 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -4580,13 +4580,13 @@ def go_deps(): name = "com_github_lance6716_pebble", build_file_proto_mode = "disable_global", importpath = "github.com/lance6716/pebble", - sha256 = "fad807913eaf5ff14651929475dda5e5c6f6838dbaeef11b7eddefe771f01f1e", - strip_prefix = "github.com/lance6716/pebble@v0.0.0-20241104073946-6f55c09bd183", + sha256 = "8fe7ce7009c4d2b0ae28f4c3d62f38b6256ee20e47944490b58cb1d33a155725", + strip_prefix = "github.com/lance6716/pebble@v0.0.0-20241108073934-da961314c63f", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241104073946-6f55c09bd183.zip", - "http://ats.apps.svc/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241104073946-6f55c09bd183.zip", - "https://cache.hawkingrei.com/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241104073946-6f55c09bd183.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241104073946-6f55c09bd183.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241108073934-da961314c63f.zip", + "http://ats.apps.svc/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241108073934-da961314c63f.zip", + "https://cache.hawkingrei.com/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241108073934-da961314c63f.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/lance6716/pebble/com_github_lance6716_pebble-v0.0.0-20241108073934-da961314c63f.zip", ], ) go_repository( diff --git a/go.mod b/go.mod index 30fe24c41c0f3..abc3f53956022 100644 --- a/go.mod +++ b/go.mod @@ -73,7 +73,7 @@ require ( github.com/kisielk/errcheck v1.7.0 github.com/klauspost/compress v1.17.9 github.com/ks3sdklib/aws-sdk-go v1.2.9 - github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183 + github.com/lance6716/pebble v0.0.0-20241108073934-da961314c63f github.com/lestrrat-go/jwx/v2 v2.0.21 github.com/mgechev/revive v1.4.0 github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 diff --git a/go.sum b/go.sum index e7a88703e4ddd..d29d71107fd3a 100644 --- a/go.sum +++ b/go.sum @@ -558,8 +558,8 @@ github.com/ks3sdklib/aws-sdk-go v1.2.9 h1:Eg0fM56r4Gjp9PiK1Bg9agJUxCAWCk236qq9DI github.com/ks3sdklib/aws-sdk-go v1.2.9/go.mod h1:xBNbOrxSnd36AQpZ8o99mGGu+blblUd9rI0MKGmeufo= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= -github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183 h1:CrFmpCAT5PGMgmQadTa2lXZrjuvpknONB1/pyxiyDsM= -github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183/go.mod h1:ZxnWA3Ab0ufDIyppyzL16j6HFNpdXeiU/1cE4Wlv/lQ= +github.com/lance6716/pebble v0.0.0-20241108073934-da961314c63f h1:KyBFXtlZDJTrwuD3wBHGk+TtdOpCB17AEm0X4eqhRdg= +github.com/lance6716/pebble v0.0.0-20241108073934-da961314c63f/go.mod h1:ZxnWA3Ab0ufDIyppyzL16j6HFNpdXeiU/1cE4Wlv/lQ= github.com/lestrrat-go/blackmagic v1.0.2 h1:Cg2gVSc9h7sz9NOByczrbUvLopQmXrfFx//N+AkAr5k= github.com/lestrrat-go/blackmagic v1.0.2/go.mod h1:UrEqBzIR2U6CnzVyUtfM6oZNMt/7O7Vohk2J0OGSAtU= github.com/lestrrat-go/httpcc v1.0.1 h1:ydWCStUeJLkpYyjLDHihupbn2tYmZ7m22BGkcvZZrIE= diff --git a/lightning/tests/config/pd-no-tls.toml b/lightning/tests/config/pd-no-tls.toml new file mode 100644 index 0000000000000..28d5da286680d --- /dev/null +++ b/lightning/tests/config/pd-no-tls.toml @@ -0,0 +1,2 @@ +[replication] +enable-placement-rules = true diff --git a/lightning/tests/config/tikv-no-tls.toml b/lightning/tests/config/tikv-no-tls.toml new file mode 100644 index 0000000000000..f38a7598c1de2 --- /dev/null +++ b/lightning/tests/config/tikv-no-tls.toml @@ -0,0 +1,23 @@ +# config of tikv +[storage] +reserve-space = "1KB" +data-dir = "/tmp/lightning_test/tikv1/" + +[coprocessor] +region-max-keys = 100 +region-split-keys = 60 + +[rocksdb] +max-open-files = 4096 +[raftdb] +max-open-files = 4096 + +[raftstore] +# true (default value) for high reliability, this can prevent data loss when power failure. +sync-log = false +capacity = "10GB" +# Speed up TiKV region heartbeat +pd-heartbeat-tick-interval = "1s" + +[cdc] +hibernate-regions-compatible=false diff --git a/lightning/tests/run_local_sst_test.sh b/lightning/tests/run_local_sst_test.sh new file mode 100755 index 0000000000000..9c585d5ea38c1 --- /dev/null +++ b/lightning/tests/run_local_sst_test.sh @@ -0,0 +1,71 @@ +#!/bin/bash +# +# Copyright 2019 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# currently the script is WIP and not used in the CI. + +set -eu +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +export UTILS_DIR="$CUR/../../tests/_utils" +export PATH="$PATH:$CUR/../../bin:$UTILS_DIR" +export TEST_DIR=/tmp/lightning_test +export COV_DIR="/tmp/group_cover" +mkdir -p $COV_DIR || true +export TIKV_CONFIG="$CUR/config/tikv-no-tls.toml" +export PD_CONFIG="$CUR/config/pd-no-tls.toml" +export TESTS_ROOT="$CUR" +source $UTILS_DIR/run_services + +export TIKV_COUNT=1 +export PD_HTTP_PROTO="http" + +# Create COV_DIR if not exists +if [ -d "$COV_DIR" ]; then + mkdir -p $COV_DIR +fi + +# Reset TEST_DIR +rm -rf $TEST_DIR && mkdir -p $TEST_DIR + +trap stop_services EXIT +start_services $@ --no-tiflash --no-tidb + +rm /tmp/*.sst || true + +# change to project root +cd $CUR/../.. +go test ./pkg/lightning/tikv -tikv-write-test -test.v -test.run TestIntegrationTest +cp /tmp/lightning_test/tikv1/import/*_write_*.sst /tmp/tikv-write-cf.sst + +for prefix in go tikv; do + bin/tikv-ctl sst_dump --file=/tmp/$prefix-write-cf.sst --command=scan --output_hex --show_properties \ + > /tmp/$prefix-write-cf-scan.txt + awk " + /from \[\] to \[\]/ { f1 = 1; next } + /Table Properties:/ { f1 = 0; f2 = 1; next } + f1 { print > \"/tmp/$prefix-write-cf-data.txt\" } + f2 { print > \"/tmp/$prefix-write-cf-properties.txt\" } + " /tmp/$prefix-write-cf-scan.txt + # filter some properties that are not deterministic by logical content + grep -v -F -e "data block size" -e "index block size" -e "entries for filter" -e "(estimated) table size" \ + -e "DB identity" -e "DB session identity" -e "DB host id" -e "original file number" -e "unique ID" \ + /tmp/$prefix-write-cf-properties.txt > /tmp/$prefix-write-cf-properties.txt.filtered +done + +diff /tmp/tikv-write-cf-data.txt /tmp/go-write-cf-data.txt +diff /tmp/tikv-write-cf-properties.txt.filtered /tmp/go-write-cf-properties.txt.filtered + +# clean tikv-ctl temporary files +rm -rf ctl-engine-info-log || true diff --git a/pkg/lightning/tikv/BUILD.bazel b/pkg/lightning/tikv/BUILD.bazel index cae38c320e978..b658db8abd11e 100644 --- a/pkg/lightning/tikv/BUILD.bazel +++ b/pkg/lightning/tikv/BUILD.bazel @@ -47,16 +47,16 @@ go_test( "local_sst_writer_test.go", "tikv_test.go", ], + data = glob(["sst-examples/**"]), embed = [":tikv"], flaky = True, shard_count = 7, deps = [ - "@com_github_cockroachdb_pebble//:pebble", - "@com_github_cockroachdb_pebble//sstable", - "@com_github_cockroachdb_pebble//vfs", "@com_github_coreos_go_semver//semver", "@com_github_google_uuid//:uuid", + "@com_github_lance6716_pebble//:pebble", "@com_github_lance6716_pebble//sstable", + "@com_github_lance6716_pebble//vfs", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/import_sstpb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", diff --git a/pkg/lightning/tikv/local_sst_writer.go b/pkg/lightning/tikv/local_sst_writer.go index 73ee591ca17ca..2557d3011c828 100644 --- a/pkg/lightning/tikv/local_sst_writer.go +++ b/pkg/lightning/tikv/local_sst_writer.go @@ -38,7 +38,6 @@ type writeCFWriter struct { func newWriteCFWriter( sstPath string, ts uint64, - identity *rockssst.Identity, ) (*writeCFWriter, error) { f, err := vfs.Default.Create(sstPath) if err != nil { @@ -46,7 +45,8 @@ func newWriteCFWriter( } writable := objstorageprovider.NewFileWritable(f) writer := rockssst.NewWriter(writable, rockssst.WriterOptions{ - // TODO(lance6716): should read TiKV config to know compression algorithm. + // TODO(lance6716): should read TiKV config to know these values. + BlockSize: 32 * 1024, Compression: rocks.ZstdCompression, // TODO(lance6716): should check the behaviour is the exactly same. FilterPolicy: rocksbloom.FilterPolicy(10), @@ -63,7 +63,7 @@ func newWriteCFWriter( return mockCollector{name: "BlobFileSizeCollector"} }, }, - }, identity) + }) return &writeCFWriter{sstWriter: writer, ts: ts}, nil } diff --git a/pkg/lightning/tikv/local_sst_writer_test.go b/pkg/lightning/tikv/local_sst_writer_test.go index 522d541e1cb4d..8bfd361aa5cfc 100644 --- a/pkg/lightning/tikv/local_sst_writer_test.go +++ b/pkg/lightning/tikv/local_sst_writer_test.go @@ -16,15 +16,16 @@ package tikv import ( "context" - "encoding/json" - "strings" + "flag" + "fmt" + "slices" "testing" + "time" - "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/sstable" - "github.com/cockroachdb/pebble/vfs" "github.com/google/uuid" + rocks "github.com/lance6716/pebble" rockssst "github.com/lance6716/pebble/sstable" + "github.com/lance6716/pebble/vfs" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -34,12 +35,39 @@ import ( "google.golang.org/grpc/credentials/insecure" ) -var ( - sortedKVs = [][2][]byte{ - {[]byte("a"), []byte("1")}, +type testCase struct { + sortedKVs [][2][]byte + ts uint64 + + expectedFilePath string +} + +var testCases []*testCase + +func init() { + testCases = make([]*testCase, 0, 2) + + testCases = append(testCases, &testCase{ + sortedKVs: [][2][]byte{ + {[]byte("a"), []byte("1")}, + }, + ts: 1, + expectedFilePath: "sst-examples/0.sst", + }) + + moreKeys := make([][2][]byte, 10000) + for i := range moreKeys { + moreKeys[i] = [2][]byte{ + []byte("key" + fmt.Sprintf("%09d", i)), + []byte("1"), + } } - ts uint64 = 1 -) + testCases = append(testCases, &testCase{ + sortedKVs: moreKeys, + ts: 404411537129996288, + expectedFilePath: "sst-examples/1.sst", + }) +} // write2ImportService4Test writes these sorted key-value pairs to the TiKV // cluster. SST files are generated by TiKV and saved in import directory if no @@ -143,28 +171,45 @@ func write2ImportService4Test( return resp.Metas, nil } -func TestGRPCWriteToTiKV(t *testing.T) { - t.Skip(`This is a manual test. You can use tiup playground and run this test. -After the test is finished, find the SST files in the import directory of the TiKV node.`) +var tikvWriteTest = flag.Bool("tikv-write-test", false, "run TestIntegrationTest") + +func TestIntegrationTest(t *testing.T) { + if !*tikvWriteTest { + t.Skip(`This is a manual test. You can use tiup playground and run this test. After the test is finished, find the SST files in the import directory of the TiKV node.`) + } ctx := context.Background() pdAddrs := []string{"127.0.0.1:2379"} + sortedKVs := make([][2][]byte, 1_000_000) + for i := range sortedKVs { + sortedKVs[i] = [2][]byte{ + []byte("key" + fmt.Sprintf("%09d", i)), + []byte("1"), + } + } + ts := uint64(404411537129996288) + sstPath := "/tmp/go-write-cf.sst" + now := time.Now() + pebbleWriteSST(t, sstPath, sortedKVs, ts) + t.Logf("write to SST takes %v", time.Since(now)) + + now = time.Now() metas, err := write2ImportService4Test(ctx, pdAddrs, sortedKVs, ts) + t.Logf("write to TiKV takes %v", time.Since(now)) require.NoError(t, err) for _, meta := range metas { t.Logf("meta UUID: %v", uuid.UUID(meta.Uuid).String()) } } -func TestPebbleWriteSST(t *testing.T) { - sstPath := "/tmp/test-write.sst" - writer, err := newWriteCFWriter(sstPath, ts, &rockssst.Identity{ - DB: "SST Writer", - Host: "lance6716-nuc10i7fnh", - Session: "DS38NDUWK5HLG8SSL5M7", - OriginalFileNumber: 1, - }) +func pebbleWriteSST( + t *testing.T, + path string, + sortedKVs [][2][]byte, + ts uint64, +) { + writer, err := newWriteCFWriter(path, ts) require.NoError(t, err) for _, kv := range sortedKVs { @@ -174,6 +219,21 @@ func TestPebbleWriteSST(t *testing.T) { err = writer.close() require.NoError(t, err) +} + +func TestPebbleWriteSST(t *testing.T) { + for i, c := range testCases { + t.Logf("start test case %d", i) + testPebbleWriteSST(t, c) + } +} + +func testPebbleWriteSST( + t *testing.T, + c *testCase, +) { + sstPath := "/tmp/test-write.sst" + pebbleWriteSST(t, sstPath, c.sortedKVs, c.ts) f, err := vfs.Default.Open(sstPath) require.NoError(t, err) @@ -183,45 +243,100 @@ func TestPebbleWriteSST(t *testing.T) { require.NoError(t, err) defer reader.Close() - layout, err := reader.Layout() + goSSTKVs, goSSTProperties := getData2Compare(t, reader) + require.Len(t, goSSTKVs, len(c.sortedKVs)) + + f2, err := vfs.Default.Open(c.expectedFilePath) + require.NoError(t, err) + readable2, err := rockssst.NewSimpleReadable(f2) + require.NoError(t, err) + reader2, err := rockssst.NewReader(readable2, rockssst.ReaderOptions{}) require.NoError(t, err) + defer reader2.Close() - infos := layout.BlockInfos(reader) - expected := ` -[ - {"Offset":0,"Length":42,"Name":"data","Compression":0,"Checksum":2258416982}, - {"Offset":121,"Length":39,"Name":"index","Compression":0,"Checksum":3727189474}, - {"Offset":165,"Length":1253,"Name":"properties","Compression":0,"Checksum":561778464}, - {"Offset":1423,"Length":79,"Name":"meta-index","Compression":0,"Checksum":955781521}, - {"Offset":1507,"Length":53,"Name":"footer","Compression":0,"Checksum":0} -]` - var expectedInfos []*rockssst.BlockInfo - err = json.Unmarshal([]byte(expected), &expectedInfos) + tikvSSTKVs, tikvSSTProperties := getData2Compare(t, reader2) + + require.Equal(t, len(tikvSSTKVs), len(goSSTKVs)) + for i, kv := range goSSTKVs { + require.Equal(t, kv[0], tikvSSTKVs[i][0], "key mismatch. index: %d", i) + require.Equal(t, kv[1], tikvSSTKVs[i][1], "value mismatch. index: %d", i) + } + require.Equal(t, tikvSSTProperties, goSSTProperties) +} + +func getData2Compare( + t *testing.T, + reader *rockssst.Reader, +) (kvs [][2][]byte, properties *rockssst.Properties) { + iter, err := reader.NewIter(nil, nil) require.NoError(t, err) - require.Equal(t, expectedInfos, infos) + defer iter.Close() + + realKVs := make([][2][]byte, 0, 10240) + + k, v := iter.First() + require.NotNil(t, k) + getKey := func(k *rocks.InternalKey) []byte { + return slices.Clone(k.UserKey) + } + getValue := func(v rocks.LazyValue) []byte { + realV, callerOwned, err2 := v.Value(nil) + require.NoError(t, err2) + if !callerOwned { + realV = slices.Clone(realV) + } + return realV + } + realKVs = append(realKVs, [2][]byte{getKey(k), getValue(v)}) + for { + k, v = iter.Next() + if k == nil { + break + } + realKVs = append(realKVs, [2][]byte{getKey(k), getValue(v)}) + } + + p := reader.Properties.Clone() + + // delete the identity properties + delete(p.UserProperties, "rocksdb.creating.db.identity") + delete(p.UserProperties, "rocksdb.creating.host.identity") + delete(p.UserProperties, "rocksdb.creating.session.identity") + delete(p.UserProperties, "rocksdb.original.file.number") + + // delete some mismatch properties because compress layer has different behaviour + p.DataSize = 0 + p.NumDataBlocks = 0 + p.IndexSize = 0 + + // TODO(lance6716): check why it's different, can we tune bloomfilter to get the + // same behaviour? + p.FilterSize = 0 + delete(p.UserProperties, "rocksdb.num.filter_entries") + + // TODO(lance6716): in integration tests we need to check + // rocksdb.tail.start.offset equals to rocksdb.data.size + delete(p.UserProperties, "rocksdb.tail.start.offset") + p.Loaded = nil + + return realKVs, p } func TestDebugReadSST(t *testing.T) { t.Skip("this is a manual test") - sstPath := "/tmp/test-write.sst" + sstPath := "/tmp/test.sst" t.Logf("read sst: %s", sstPath) f, err := vfs.Default.Open(sstPath) require.NoError(t, err) - readable, err := sstable.NewSimpleReadable(f) + readable, err := rockssst.NewSimpleReadable(f) require.NoError(t, err) - reader, err := sstable.NewReader(readable, sstable.ReaderOptions{}) + reader, err := rockssst.NewReader(readable, rockssst.ReaderOptions{}) require.NoError(t, err) defer reader.Close() - layout, err := reader.Layout() - require.NoError(t, err) - - content := &strings.Builder{} - layout.Describe(content, true, reader, nil) - - t.Logf("layout:\n %s", content.String()) t.Logf("properties:\n %s", reader.Properties.String()) + t.SkipNow() iter, err := reader.NewIter(nil, nil) require.NoError(t, err) @@ -231,7 +346,7 @@ func TestDebugReadSST(t *testing.T) { if k == nil { return } - getValue := func(v pebble.LazyValue) []byte { + getValue := func(v rocks.LazyValue) []byte { realV, _, err2 := v.Value(nil) require.NoError(t, err2) return realV diff --git a/pkg/lightning/tikv/sst-examples/0.sst b/pkg/lightning/tikv/sst-examples/0.sst new file mode 100644 index 0000000000000..0144af0a39744 Binary files /dev/null and b/pkg/lightning/tikv/sst-examples/0.sst differ diff --git a/pkg/lightning/tikv/sst-examples/1.sst b/pkg/lightning/tikv/sst-examples/1.sst new file mode 100644 index 0000000000000..5e53390a5c06d Binary files /dev/null and b/pkg/lightning/tikv/sst-examples/1.sst differ diff --git a/tests/_utils/run_services b/tests/_utils/run_services index 5c0fa16d6408c..617821cfc7577 100644 --- a/tests/_utils/run_services +++ b/tests/_utils/run_services @@ -16,6 +16,7 @@ set -eu +export PD_HTTP_PROTO="https" export PD_PEER_ADDR="127.0.0.1:2380" export PD_ADDR="127.0.0.1:2379" export PD_PID="${TEST_DIR:?}/pd_pid.txt" @@ -79,9 +80,10 @@ start_services() { start_pd() { echo "Starting PD..." mkdir -m 700 -p "$TEST_DIR/pd" + pd-server \ - --client-urls "https://$PD_ADDR" \ - --peer-urls "https://$PD_PEER_ADDR" \ + --client-urls "$PD_HTTP_PROTO://$PD_ADDR" \ + --peer-urls "$PD_HTTP_PROTO://$PD_PEER_ADDR" \ --log-file "$TEST_DIR/pd.log" \ --data-dir "$TEST_DIR/pd" \ --config $PD_CONFIG & @@ -89,7 +91,7 @@ start_pd() { echo -e "$pid" > "${PD_PID}" # wait until PD is online... i=0 - while ! run_curl "https://$PD_ADDR/pd/api/v1/version"; do + while ! run_curl "$PD_HTTP_PROTO://$PD_ADDR/pd/api/v1/version"; do i=$((i+1)) if [ "$i" -gt 20 ]; then echo 'Failed to start PD' @@ -158,7 +160,7 @@ start_tikv() { ensure_tikv() { echo "Waiting initializing TiKV..." - while ! run_curl "https://$PD_ADDR/pd/api/v1/cluster/status" | grep '"is_initialized": true'; do + while ! run_curl "$PD_HTTP_PROTO://$PD_ADDR/pd/api/v1/cluster/status" | grep '"is_initialized": true'; do i=$((i+1)) if [ "$i" -gt 20 ]; then echo 'Failed to initialize TiKV cluster' @@ -196,6 +198,7 @@ start_services_impl() { cleanup_data || true RUN_TIFLASH=true + RUN_TIDB=true while [[ $# -gt 0 ]] do @@ -211,6 +214,10 @@ start_services_impl() { RUN_TIFLASH=false shift # past argument ;; + --no-tidb) + RUN_TIDB=false + shift # past argument + ;; *) # unknown option echo "Unknown args $1" exit 1 @@ -227,14 +234,16 @@ start_services_impl() { start_tikv "$i" done ensure_tikv - start_tidb + if $RUN_TIDB; then + start_tidb + fi if $RUN_TIFLASH; then start_tiflash fi i=0 - while ! run_curl "https://$PD_ADDR/pd/api/v1/cluster/status" | grep -q "\"is_initialized\": true"; do + while ! run_curl "$PD_HTTP_PROTO://$PD_ADDR/pd/api/v1/cluster/status" | grep -q "\"is_initialized\": true"; do i=$((i+1)) if [ "$i" -gt 20 ]; then echo 'Failed to bootstrap cluster'