From 86af5550c6b5026b15796a9873e5f88464620169 Mon Sep 17 00:00:00 2001 From: reshke Date: Mon, 8 Jul 2024 17:05:15 +0500 Subject: [PATCH] Change yacc and protos for multidim key ranges. (#545) * Change yacc and protos for multidim key ranges. --- balancer/provider/balancer.go | 59 +- cmd/mover/main.go | 24 +- cmd/spqrdump/main.go | 21 +- coordinator/provider/coordinator.go | 177 +++--- coordinator/provider/keyranges.go | 9 +- docker-compose.yaml | 4 +- docker/coordinator/Dockerfile | 2 +- examples/balancer.yaml | 24 +- examples/coordinator.yaml | 2 +- examples/shard_data.yaml | 13 + pkg/clientinteractor/interactor.go | 8 +- pkg/coord/adapter.go | 42 +- pkg/coord/local/clocal.go | 114 +++- pkg/coord/local/clocal_test.go | 82 +++ pkg/datatransfers/data_transfers.go | 31 +- pkg/decode/spqrql.go | 5 +- pkg/decode/spqrql_test.go | 30 +- pkg/meta/meta.go | 13 +- pkg/models/hashfunction/hashfunction.go | 67 +- pkg/models/kr/keyrange.go | 369 ++++++++--- pkg/models/kr/keyrange_test.go | 20 +- pkg/models/kr/keyrangemgr.go | 2 +- pkg/protos/key_range.pb.go | 295 +++++---- protos/key_range.proto | 7 +- qdb/etcdqdb.go | 4 +- qdb/memqdb_test.go | 10 +- qdb/models.go | 16 +- qdb/ops/ops.go | 10 +- router/client/client.go | 2 +- router/grpc/qrouter.go | 8 +- router/instance/etcd.go | 2 +- router/mock/qrouter/mock_qrouter.go | 2 +- router/qrouter/proxy_routing.go | 341 ++++++---- router/qrouter/proxy_routing_test.go | 597 +++++++++++++----- router/qrouter/qrouter.go | 3 +- router/relay/qstate.go | 7 +- test/drivers/gorm-regress/docker-compose.yaml | 2 +- .../hibernate-regress/docker-compose.yaml | 2 +- test/drivers/jdbc-regress/docker-compose.yaml | 2 +- test/feature/docker-compose.yaml | 4 +- test/feature/features/move.feature | 4 +- test/feature/features/spqrdump.feature | 8 +- test/feature/spqr_test.go | 8 +- test/regress/docker-compose.yaml | 2 +- .../console/expected/delete_distribution.out | 6 +- .../tests/console/sql/delete_distribution.sql | 4 +- .../router/expected/alter_distribution.out | 6 +- .../tests/router/expected/copy_routing.out | 2 +- .../tests/router/expected/hash_routing.out | 4 +- .../tests/router/expected/mixed_routing.out | 12 +- .../tests/router/sql/alter_distribution.sql | 7 +- .../regress/tests/router/sql/copy_routing.sql | 2 +- .../regress/tests/router/sql/hash_routing.sql | 2 +- test/stress/docker-compose.yaml | 2 +- test/xproto/proto_test.go | 149 +++++ yacc/console/ast.go | 8 +- yacc/console/gram.go | 591 ++++++++--------- yacc/console/gram.y | 71 ++- yacc/console/yx_test.go | 50 +- 59 files changed, 2295 insertions(+), 1075 deletions(-) create mode 100644 examples/shard_data.yaml create mode 100644 pkg/coord/local/clocal_test.go diff --git a/balancer/provider/balancer.go b/balancer/provider/balancer.go index 08db930d1..cc2b28797 100644 --- a/balancer/provider/balancer.go +++ b/balancer/provider/balancer.go @@ -2,8 +2,10 @@ package provider import ( "context" + "encoding/binary" "fmt" "sort" + "strconv" "strings" "github.com/google/uuid" @@ -15,6 +17,7 @@ import ( "github.com/pg-sharding/spqr/pkg/models/tasks" protos "github.com/pg-sharding/spqr/pkg/protos" "github.com/pg-sharding/spqr/pkg/spqrlog" + "github.com/pg-sharding/spqr/qdb" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" ) @@ -347,10 +350,11 @@ func (b *BalancerImpl) getKRCondition(rel *distributions.DistributedRelation, kR } else { hashedCol = entry.Column } + // TODO: fix multidim case if nextKR != nil { - buf[i] = fmt.Sprintf("%s >= %s AND %s < %s", hashedCol, string(kRange.LowerBound), hashedCol, string(nextKR.LowerBound)) + buf[i] = fmt.Sprintf("%s >= %s AND %s < %s", hashedCol, kRange.SendRaw()[0], hashedCol, nextKR.SendRaw()[0]) } else { - buf[i] = fmt.Sprintf("%s >= %s", hashedCol, string(kRange.LowerBound)) + buf[i] = fmt.Sprintf("%s >= %s", hashedCol, kRange.SendRaw()[0]) } } return strings.Join(buf, " AND "), nil @@ -501,6 +505,7 @@ func (b *BalancerImpl) getTasks(ctx context.Context, shardFrom *ShardMetrics, kr maxCount = count } } + var rel *distributions.DistributedRelation = nil allRels, err := b.getKRRelations(ctx, b.dsToKeyRanges[ds][krInd]) if err != nil { @@ -516,6 +521,16 @@ func (b *BalancerImpl) getTasks(ctx context.Context, shardFrom *ShardMetrics, kr return nil, fmt.Errorf("relation \"%s\" not found", relName) } + dsService := protos.NewDistributionServiceClient(b.coordinatorConn) + + dsS, err := dsService.GetDistribution(ctx, &protos.GetDistributionRequest{ + Id: ds, + }) + + if err != nil { + return nil, err + } + moveCount := min((keyCount+config.BalancerConfig().KeysPerMove-1)/config.BalancerConfig().KeysPerMove, config.BalancerConfig().MaxMoveCount) counts := make([]int, moveCount) @@ -552,12 +567,38 @@ func (b *BalancerImpl) getTasks(ctx context.Context, shardFrom *ShardMetrics, kr if err := row.Scan(&idx); err != nil { return nil, err } + + var bound []byte + + switch dsS.Distribution.ColumnTypes[0] { + case qdb.ColumnTypeVarchar: + fallthrough + case qdb.ColumnTypeVarcharDeprecated: + bound = []byte(idx) + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeInteger: + i, err := strconv.ParseInt(idx, 10, 64) + if err != nil { + return nil, err + } + bound = make([]byte, 8) + binary.PutVarint(bound, i) + case qdb.ColumnTypeUinteger: + i, err := strconv.ParseUint(idx, 10, 64) + if err != nil { + return nil, err + } + bound = make([]byte, 8) + binary.PutUvarint(bound, i) + } + groupTasks[len(groupTasks)-1-i] = &tasks.Task{ ShardFromId: shardFrom.ShardId, ShardToId: shardToId, KrIdFrom: krId, KrIdTo: krIdTo, - Bound: []byte(idx), + Bound: bound, } totalCount += count } @@ -667,6 +708,7 @@ func (b *BalancerImpl) executeTasks(ctx context.Context, group *tasks.TaskGroup) func (b *BalancerImpl) updateKeyRanges(ctx context.Context) error { keyRangeService := protos.NewKeyRangeServiceClient(b.coordinatorConn) + distrService := protos.NewDistributionServiceClient(b.coordinatorConn) keyRangesProto, err := keyRangeService.ListAllKeyRanges(ctx, &protos.ListAllKeyRangesRequest{}) if err != nil { return err @@ -676,11 +718,17 @@ func (b *BalancerImpl) updateKeyRanges(ctx context.Context) error { if _, ok := keyRanges[krProto.DistributionId]; !ok { keyRanges[krProto.DistributionId] = make([]*kr.KeyRange, 0) } - keyRanges[krProto.DistributionId] = append(keyRanges[krProto.DistributionId], kr.KeyRangeFromProto(krProto)) + ds, err := distrService.GetDistribution(ctx, &protos.GetDistributionRequest{ + Id: krProto.DistributionId, + }) + if err != nil { + return err + } + keyRanges[krProto.DistributionId] = append(keyRanges[krProto.DistributionId], kr.KeyRangeFromProto(krProto, ds.Distribution.ColumnTypes)) } for _, krs := range keyRanges { sort.Slice(krs, func(i, j int) bool { - return kr.CmpRangesLess(krs[i].LowerBound, krs[j].LowerBound) + return kr.CmpRangesLess(krs[i].LowerBound, krs[j].LowerBound, krs[j].ColumnTypes) }) } @@ -688,6 +736,7 @@ func (b *BalancerImpl) updateKeyRanges(ctx context.Context) error { b.dsToKrIdx = make(map[string]map[string]int) b.shardKr = make(map[string][]string) b.krToDs = make(map[string]string) + for ds, krs := range b.dsToKeyRanges { for i, krg := range krs { b.krToDs[krg.ID] = ds diff --git a/cmd/mover/main.go b/cmd/mover/main.go index 59db82d80..80f1037a5 100644 --- a/cmd/mover/main.go +++ b/cmd/mover/main.go @@ -4,11 +4,12 @@ import ( "context" "flag" "fmt" - "github.com/pg-sharding/spqr/pkg/models/distributions" "io" "os" "strings" + "github.com/pg-sharding/spqr/pkg/models/distributions" + "github.com/jackc/pgx/v5" _ "github.com/lib/pq" "github.com/pg-sharding/spqr/pkg/models/kr" @@ -103,10 +104,10 @@ FROM information_schema.tables; // TODO: support multi-column move in SPQR2 if nextKeyRange == nil { qry = fmt.Sprintf("copy (delete from %s WHERE %s >= %s returning *) to stdout", rel.Name, - rel.DistributionKey[0].Column, keyRange.LowerBound) + rel.DistributionKey[0].Column, keyRange.SendRaw()[0]) } else { qry = fmt.Sprintf("copy (delete from %s WHERE %s >= %s and %s < %s returning *) to stdout", rel.Name, - rel.DistributionKey[0].Column, keyRange.LowerBound, rel.DistributionKey[0].Column, nextKeyRange.LowerBound) + rel.DistributionKey[0].Column, keyRange.SendRaw()[0], rel.DistributionKey[0].Column, nextKeyRange.SendRaw()[0]) } spqrlog.Zero.Debug(). @@ -132,6 +133,7 @@ FROM information_schema.tables; spqrlog.Zero.Debug().Msg("copy cmd executed") } + /* TODO: handle errors here */ _ = txTo.Commit(ctx) _ = txFrom.Commit(ctx) return nil @@ -165,7 +167,14 @@ func main() { spqrlog.Zero.Error().Err(err).Msg("") return } - keyRange := kr.KeyRangeFromDB(qdbKr) + + ds, err := db.GetDistribution(ctx, qdbKr.DistributionId) + if err != nil { + spqrlog.Zero.Error().Err(err).Msg("") + return + } + + keyRange := kr.KeyRangeFromDB(qdbKr, ds.ColTypes) krs, err := db.ListKeyRanges(ctx, keyRange.Distribution) if err != nil { @@ -176,9 +185,10 @@ func main() { var nextKeyRange *kr.KeyRange for _, currkr := range krs { - if kr.CmpRangesLess(keyRange.LowerBound, currkr.LowerBound) { - if nextKeyRange == nil || kr.CmpRangesLess(currkr.LowerBound, nextKeyRange.LowerBound) { - nextKeyRange = kr.KeyRangeFromDB(currkr) + typedKr := kr.KeyRangeFromDB(currkr, ds.ColTypes) + if kr.CmpRangesLess(keyRange.LowerBound, typedKr.LowerBound, ds.ColTypes) { + if nextKeyRange == nil || kr.CmpRangesLess(typedKr.LowerBound, nextKeyRange.LowerBound, ds.ColTypes) { + nextKeyRange = typedKr } } } diff --git a/cmd/spqrdump/main.go b/cmd/spqrdump/main.go index 87e2ef1c3..f0d5a866d 100644 --- a/cmd/spqrdump/main.go +++ b/cmd/spqrdump/main.go @@ -12,6 +12,7 @@ import ( "google.golang.org/grpc" "github.com/pg-sharding/spqr/pkg/conn" + "github.com/pg-sharding/spqr/pkg/models/kr" "github.com/pg-sharding/spqr/pkg/spqrlog" "github.com/pg-sharding/spqr/pkg/decode" @@ -116,14 +117,16 @@ func getconn() (*pgproto3.Frontend, error) { // TODO : unit tests func DumpKeyRangesPsql() error { return dumpPsql("SHOW key_ranges;", func(v *pgproto3.DataRow) (string, error) { - l := string(v.Values[2]) + l := v.Values[2] id := string(v.Values[0]) shard := string(v.Values[1]) return decode.KeyRange( - &protos.KeyRangeInfo{ - KeyRange: &protos.KeyRange{LowerBound: l}, - ShardId: shard, Krid: id}), nil + &kr.KeyRange{ + LowerBound: []interface{}{l}, + ID: id, + ShardID: shard, + }), nil }) } @@ -171,13 +174,21 @@ func DumpKeyRanges() error { } rCl := protos.NewKeyRangeServiceClient(cc) + dCl := protos.NewDistributionServiceClient(cc) if keys, err := rCl.ListAllKeyRanges(context.Background(), &protos.ListAllKeyRangesRequest{}); err != nil { spqrlog.Zero.Error(). Err(err). Msg("failed to dump endpoint rules") } else { for _, krg := range keys.KeyRangesInfo { - fmt.Println(decode.KeyRange(krg)) + ds, err := dCl.GetDistribution(context.Background(), &protos.GetDistributionRequest{ + Id: krg.DistributionId, + }) + if err != nil { + return err + } + krCurr := kr.KeyRangeFromProto(krg, ds.Distribution.ColumnTypes) + fmt.Println(decode.KeyRange(krCurr)) } } diff --git a/coordinator/provider/coordinator.go b/coordinator/provider/coordinator.go index 3ed548eac..61429d36b 100644 --- a/coordinator/provider/coordinator.go +++ b/coordinator/provider/coordinator.go @@ -285,7 +285,7 @@ func NewCoordinator(tlsconfig *tls.Config, db qdb.XQDB) *qdbCoordinator { // TODO : unit tests func (qc *qdbCoordinator) lockCoordinator(ctx context.Context, initialRouter bool) bool { - registerRouter := func() bool { + updateCoordinator := func() bool { if !initialRouter { return true } @@ -297,9 +297,7 @@ func (qc *qdbCoordinator) lockCoordinator(ctx context.Context, initialRouter boo if err := qc.RegisterRouter(ctx, router); err != nil { spqrlog.Zero.Error().Err(err).Msg("register router when locking coordinator") } - if err := qc.SyncRouterMetadata(ctx, router); err != nil { - spqrlog.Zero.Error().Err(err).Msg("sync router metadata when locking coordinator") - } + if err := qc.UpdateCoordinator(ctx, net.JoinHostPort(config.CoordinatorConfig().Host, config.CoordinatorConfig().GrpcApiPort)); err != nil { return false } @@ -313,7 +311,7 @@ func (qc *qdbCoordinator) lockCoordinator(ctx context.Context, initialRouter boo return false case <-time.After(time.Second): if err := qc.db.TryCoordinatorLock(context.TODO()); err == nil { - return registerRouter() + return updateCoordinator() } else { spqrlog.Zero.Error().Err(err).Msg("qdb already taken, waiting for connection") } @@ -321,7 +319,7 @@ func (qc *qdbCoordinator) lockCoordinator(ctx context.Context, initialRouter boo } } - return registerRouter() + return updateCoordinator() } // RunCoordinator side effect: it runs an asynchronous goroutine @@ -402,8 +400,6 @@ func (qc *qdbCoordinator) traverseRouters(ctx context.Context, cb func(cc *grpc. } defer cc.Close() - defer cc.Close() - if err := cb(cc); err != nil { spqrlog.Zero.Debug().Err(err).Str("router id", rtr.ID).Msg("traverse routers") return err @@ -446,7 +442,7 @@ func (qc *qdbCoordinator) AddRouter(ctx context.Context, router *topology.Router func (qc *qdbCoordinator) CreateKeyRange(ctx context.Context, keyRange *kr.KeyRange) error { // add key range to metadb spqrlog.Zero.Debug(). - Bytes("lower-bound", keyRange.LowerBound). + Bytes("lower-bound", keyRange.Raw()[0]). Str("shard-id", keyRange.ShardID). Str("key-range-id", keyRange.ID). Msg("add key range") @@ -461,15 +457,10 @@ func (qc *qdbCoordinator) CreateKeyRange(ctx context.Context, keyRange *kr.KeyRa resp, err := cl.CreateKeyRange(ctx, &routerproto.CreateKeyRangeRequest{ KeyRangeInfo: keyRange.ToProto(), }) - - if err != nil { - return err - } - - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). Msg("add key range response") - return nil + return err }) } @@ -480,7 +471,11 @@ func (qc *qdbCoordinator) GetKeyRange(ctx context.Context, krId string) (*kr.Key if err != nil { return nil, err } - return kr.KeyRangeFromDB(krDb), nil + ds, err := qc.db.GetDistribution(ctx, krDb.DistributionId) + if err != nil { + return nil, err + } + return kr.KeyRangeFromDB(krDb, ds.ColTypes), nil } // TODO : unit tests @@ -492,7 +487,11 @@ func (qc *qdbCoordinator) ListKeyRanges(ctx context.Context, distribution string keyr := make([]*kr.KeyRange, 0, len(keyRanges)) for _, keyRange := range keyRanges { - keyr = append(keyr, kr.KeyRangeFromDB(keyRange)) + ds, err := qc.db.GetDistribution(ctx, keyRange.DistributionId) + if err != nil { + return nil, err + } + keyr = append(keyr, kr.KeyRangeFromDB(keyRange, ds.ColTypes)) } return keyr, nil @@ -507,7 +506,11 @@ func (qc *qdbCoordinator) ListAllKeyRanges(ctx context.Context) ([]*kr.KeyRange, keyr := make([]*kr.KeyRange, 0, len(keyRanges)) for _, keyRange := range keyRanges { - keyr = append(keyr, kr.KeyRangeFromDB(keyRange)) + ds, err := qc.db.GetDistribution(ctx, keyRange.DistributionId) + if err != nil { + return nil, err + } + keyr = append(keyr, kr.KeyRangeFromDB(keyRange, ds.ColTypes)) } return keyr, nil @@ -520,26 +523,29 @@ func (qc *qdbCoordinator) MoveKeyRange(ctx context.Context, keyRange *kr.KeyRang // TODO : unit tests func (qc *qdbCoordinator) LockKeyRange(ctx context.Context, keyRangeID string) (*kr.KeyRange, error) { - keyRangeDB, err := qc.db.LockKeyRange(ctx, keyRangeID) + + keyRangeDB, err := qc.QDB().LockKeyRange(ctx, keyRangeID) if err != nil { return nil, err } + ds, err := qc.QDB().GetDistribution(ctx, keyRangeDB.DistributionId) + if err != nil { + _ = qc.QDB().UnlockKeyRange(ctx, keyRangeID) + return nil, err + } - keyRange := kr.KeyRangeFromDB(keyRangeDB) + keyRange := kr.KeyRangeFromDB(keyRangeDB, ds.ColTypes) return keyRange, qc.traverseRouters(ctx, func(cc *grpc.ClientConn) error { cl := routerproto.NewKeyRangeServiceClient(cc) resp, err := cl.LockKeyRange(ctx, &routerproto.LockKeyRangeRequest{ Id: []string{keyRangeID}, }) - if err != nil { - return err - } - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). Msg("lock key range response") - return nil + return err }) } @@ -553,14 +559,12 @@ func (qc *qdbCoordinator) UnlockKeyRange(ctx context.Context, keyRangeID string) resp, err := cl.UnlockKeyRange(ctx, &routerproto.UnlockKeyRangeRequest{ Id: []string{keyRangeID}, }) - if err != nil { - return err - } - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). - Msg("lock key range response") - return nil + Msg("unlock key range response") + + return err }) } @@ -577,7 +581,7 @@ func (qc *qdbCoordinator) Split(ctx context.Context, req *kr.SplitKeyRange) erro return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "key range %v already present in qdb", req.Krid) } - krOld, err := qc.db.LockKeyRange(ctx, req.SourceID) + krOldDB, err := qc.db.LockKeyRange(ctx, req.SourceID) if err != nil { return err } @@ -588,50 +592,61 @@ func (qc *qdbCoordinator) Split(ctx context.Context, req *kr.SplitKeyRange) erro } }() - ds, err := qc.db.GetDistribution(ctx, krOld.DistributionId) + ds, err := qc.QDB().GetDistribution(ctx, krOldDB.DistributionId) + if err != nil { return err } - if kr.CmpRangesEqual(krOld.LowerBound, req.Bound) { + krOld := kr.KeyRangeFromDB(krOldDB, ds.ColTypes) + + eph := kr.KeyRangeFromBytes(req.Bound, ds.ColTypes) + + if kr.CmpRangesEqual(krOld.LowerBound, eph.LowerBound, ds.ColTypes) { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "failed to split because bound equals lower of the key range") } - if kr.CmpRangesLess(req.Bound, krOld.LowerBound) { + + if kr.CmpRangesLess(eph.LowerBound, krOld.LowerBound, ds.ColTypes) { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "failed to split because bound is out of key range") } - krs, err := qc.db.ListKeyRanges(ctx, ds.ID) + krs, err := qc.ListKeyRanges(ctx, ds.ID) if err != nil { return err } for _, kRange := range krs { - if kr.CmpRangesLess(krOld.LowerBound, kRange.LowerBound) && kr.CmpRangesLessEqual(kRange.LowerBound, req.Bound) { - return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "failed to split because bound intersects with \"%s\" key range", kRange.KeyRangeID) + if kr.CmpRangesLess(krOld.LowerBound, kRange.LowerBound, ds.ColTypes) && kr.CmpRangesLessEqual(kRange.LowerBound, eph.LowerBound, ds.ColTypes) { + return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "failed to split because bound intersects with \"%s\" key range", kRange.ID) } } - krNew := &kr.KeyRange{ - LowerBound: func() []byte { - if req.SplitLeft { - return krOld.LowerBound - } - return req.Bound - }(), - ID: req.Krid, - ShardID: krOld.ShardID, - Distribution: krOld.DistributionId, - } + krNew := kr.KeyRangeFromDB( + &qdb.KeyRange{ + // fix multidim case + LowerBound: func() [][]byte { + if req.SplitLeft { + return krOld.Raw() + } + return req.Bound + }(), + KeyRangeID: req.Krid, + ShardID: krOld.ShardID, + DistributionId: krOld.Distribution, + }, + ds.ColTypes, + ) spqrlog.Zero.Debug(). - Bytes("lower-bound", krNew.LowerBound). + Bytes("lower-bound", krNew.Raw()[0]). Str("shard-id", krNew.ShardID). Str("id", krNew.ID). Msg("new key range") if req.SplitLeft { - krOld.LowerBound = req.Bound + krOld.LowerBound = kr.KeyRangeFromBytes(req.Bound, ds.ColTypes).LowerBound } - if err := ops.ModifyKeyRangeWithChecks(ctx, qc.db, kr.KeyRangeFromDB(krOld)); err != nil { + + if err := ops.ModifyKeyRangeWithChecks(ctx, qc.db, krOld); err != nil { return err } @@ -642,13 +657,13 @@ func (qc *qdbCoordinator) Split(ctx context.Context, req *kr.SplitKeyRange) erro if err := qc.traverseRouters(ctx, func(cc *grpc.ClientConn) error { cl := routerproto.NewKeyRangeServiceClient(cc) resp, err := cl.SplitKeyRange(ctx, &routerproto.SplitKeyRangeRequest{ - Bound: req.Bound, + Bound: req.Bound[0], // fix multidim case SourceId: req.SourceID, NewId: krNew.ID, }) - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). - Msg("drop key range response") + Msg("split key range response") return err }); err != nil { return err @@ -665,7 +680,7 @@ func (qc *qdbCoordinator) DropKeyRangeAll(ctx context.Context) error { if err := qc.traverseRouters(ctx, func(cc *grpc.ClientConn) error { cl := routerproto.NewKeyRangeServiceClient(cc) resp, err := cl.DropAllKeyRanges(ctx, &routerproto.DropAllKeyRangesRequest{}) - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). Msg("drop key range response") return err @@ -686,7 +701,7 @@ func (qc *qdbCoordinator) DropKeyRange(ctx context.Context, id string) error { resp, err := cl.DropKeyRange(ctx, &routerproto.DropKeyRangeRequest{ Id: []string{id}, }) - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). Msg("drop key range response") return err @@ -700,7 +715,7 @@ func (qc *qdbCoordinator) DropKeyRange(ctx context.Context, id string) error { // TODO : unit tests func (qc *qdbCoordinator) Unite(ctx context.Context, uniteKeyRange *kr.UniteKeyRange) error { - krBase, err := qc.db.LockKeyRange(ctx, uniteKeyRange.BaseKeyRangeId) + krBaseDb, err := qc.db.LockKeyRange(ctx, uniteKeyRange.BaseKeyRangeId) if err != nil { return err } @@ -711,7 +726,7 @@ func (qc *qdbCoordinator) Unite(ctx context.Context, uniteKeyRange *kr.UniteKeyR } }() - krAppendage, err := qc.db.LockKeyRange(ctx, uniteKeyRange.AppendageKeyRangeId) + krAppendageDb, err := qc.db.LockKeyRange(ctx, uniteKeyRange.AppendageKeyRangeId) if err != nil { return err } @@ -722,44 +737,48 @@ func (qc *qdbCoordinator) Unite(ctx context.Context, uniteKeyRange *kr.UniteKeyR } }() + ds, err := qc.db.GetDistribution(ctx, krBaseDb.DistributionId) + if err != nil { + return err + } + + krBase := kr.KeyRangeFromDB(krBaseDb, ds.ColTypes) + krAppendage := kr.KeyRangeFromDB(krAppendageDb, ds.ColTypes) + if krBase.ShardID != krAppendage.ShardID { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "failed to unite key ranges routing different shards") } - if krBase.DistributionId != krAppendage.DistributionId { + if krBase.Distribution != krAppendage.Distribution { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "failed to unite key ranges of different distributions") } - ds, err := qc.db.GetDistribution(ctx, krBase.DistributionId) - if err != nil { - return err - } // TODO: check all types when composite keys are supported krLeft, krRight := krBase, krAppendage - if kr.CmpRangesLess(krRight.LowerBound, krLeft.LowerBound) { + if kr.CmpRangesLess(krRight.LowerBound, krLeft.LowerBound, ds.ColTypes) { krLeft, krRight = krRight, krLeft } - krs, err := qc.db.ListKeyRanges(ctx, ds.ID) + krs, err := qc.ListKeyRanges(ctx, ds.ID) if err != nil { return err } for _, kRange := range krs { - if kRange.KeyRangeID != krLeft.KeyRangeID && - kRange.KeyRangeID != krRight.KeyRangeID && - kr.CmpRangesLessEqual(krLeft.LowerBound, kRange.LowerBound) && - kr.CmpRangesLessEqual(kRange.LowerBound, krRight.LowerBound) { + if kRange.ID != krLeft.ID && + kRange.ID != krRight.ID && + kr.CmpRangesLessEqual(krLeft.LowerBound, kRange.LowerBound, ds.ColTypes) && + kr.CmpRangesLessEqual(kRange.LowerBound, krRight.LowerBound, ds.ColTypes) { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "failed to unite non-adjacent key ranges") } } - if err := qc.db.DropKeyRange(ctx, krAppendage.KeyRangeID); err != nil { + if err := qc.db.DropKeyRange(ctx, krAppendage.ID); err != nil { return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "failed to drop an old key range: %s", err.Error()) } - if krLeft.KeyRangeID != krBase.KeyRangeID { + if krLeft.ID != krBase.ID { krBase.LowerBound = krAppendage.LowerBound } - if err := ops.ModifyKeyRangeWithChecks(ctx, qc.db, kr.KeyRangeFromDB(krBase)); err != nil { + if err := ops.ModifyKeyRangeWithChecks(ctx, qc.db, krBase); err != nil { return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "failed to update a new key range: %s", err.Error()) } @@ -770,7 +789,7 @@ func (qc *qdbCoordinator) Unite(ctx context.Context, uniteKeyRange *kr.UniteKeyR AppendageId: uniteKeyRange.AppendageKeyRangeId, }) - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", resp). Msg("merge key range response") return err @@ -906,7 +925,7 @@ func (qc *qdbCoordinator) Move(ctx context.Context, req *kr.MoveKeyRange) error Id: krg.ID, ToShardId: krg.ShardID, }) - spqrlog.Zero.Debug(). + spqrlog.Zero.Debug().Err(err). Interface("response", moveResp). Msg("move key range response") return err @@ -992,8 +1011,12 @@ func (qc *qdbCoordinator) SyncRouterMetadata(ctx context.Context, qRouter *topol } for _, keyRange := range keyRanges { + ds, err := qc.db.GetDistribution(ctx, keyRange.DistributionId) + if err != nil { + return err + } resp, err := krClient.CreateKeyRange(ctx, &routerproto.CreateKeyRangeRequest{ - KeyRangeInfo: kr.KeyRangeFromDB(keyRange).ToProto(), + KeyRangeInfo: kr.KeyRangeFromDB(keyRange, ds.ColTypes).ToProto(), }) if err != nil { diff --git a/coordinator/provider/keyranges.go b/coordinator/provider/keyranges.go index a9ccd7658..eac497cec 100644 --- a/coordinator/provider/keyranges.go +++ b/coordinator/provider/keyranges.go @@ -40,7 +40,12 @@ func (c *CoordinatorService) DropKeyRange(ctx context.Context, request *protos.D // TODO : unit tests func (c *CoordinatorService) CreateKeyRange(ctx context.Context, request *protos.CreateKeyRangeRequest) (*protos.ModifyReply, error) { - err := c.impl.CreateKeyRange(ctx, kr.KeyRangeFromProto(request.KeyRangeInfo)) + ds, err := c.impl.GetDistribution(ctx, request.KeyRangeInfo.DistributionId) + if err != nil { + return nil, err + } + + err = c.impl.CreateKeyRange(ctx, kr.KeyRangeFromProto(request.KeyRangeInfo, ds.ColTypes)) if err != nil { return nil, err } @@ -72,7 +77,7 @@ func (c *CoordinatorService) UnlockKeyRange(ctx context.Context, request *protos // TODO : unit tests func (c *CoordinatorService) SplitKeyRange(ctx context.Context, request *protos.SplitKeyRangeRequest) (*protos.ModifyReply, error) { splitKR := &kr.SplitKeyRange{ - Bound: request.Bound, + Bound: [][]byte{request.Bound}, // fix multidim case Krid: request.NewId, SourceID: request.SourceId, SplitLeft: request.SplitLeft, diff --git a/docker-compose.yaml b/docker-compose.yaml index 26d6c52b3..4378f17d4 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -59,7 +59,7 @@ services: environment: - ROUTER_CONFIG=/spqr/docker/router/cfg.yaml - COORDINATOR_CONFIG=/spqr/docker/coordinator/cfg.yaml - - ROUTER_LOG='router.log' + - ROUTER_LOG=/var/log/spqr-router.log router2: build: dockerfile: ./docker/router/Dockerfile @@ -72,7 +72,7 @@ services: environment: - ROUTER_CONFIG=/spqr/docker/router/cfg2.yaml - COORDINATOR_CONFIG=/spqr/docker/coordinator/cfg2.yaml - - ROUTER_LOG='router.log' + - ROUTER_LOG=/var/log/spqr-router.log slicer: build: dockerfile: ./docker/slicer/Dockerfile diff --git a/docker/coordinator/Dockerfile b/docker/coordinator/Dockerfile index 21bb5a2f8..e89396471 100644 --- a/docker/coordinator/Dockerfile +++ b/docker/coordinator/Dockerfile @@ -2,4 +2,4 @@ FROM spqr-base-image RUN apt-get update && apt-get install -y postgresql-client -ENTRYPOINT /spqr/spqr-coordinator -c ${COORDINATOR_CONFIG=/spqr/docker/coordinator/cfg.yaml} \ No newline at end of file +ENTRYPOINT /spqr/spqr-coordinator -c ${COORDINATOR_CONFIG=/spqr/docker/coordinator/cfg.yaml} >> ./coord.log \ No newline at end of file diff --git a/examples/balancer.yaml b/examples/balancer.yaml index 9b5a92c8a..15afe3244 100644 --- a/examples/balancer.yaml +++ b/examples/balancer.yaml @@ -1,19 +1,13 @@ -log_level: info +log_level: debug -installation_db_name: 'db1' -installation_table_name: 'x' -installation_sharding_key: 'id' -installation_user_name: 'user1' -installation_password: '12345678' -installation_max_retries: 5 -installation_ssl_mode: 'disable' +coordinator_address: "localhost:7003" - coordinator_address: 'spqr_coordinator:7003' -coordinator_max_retries: 5 +shards_config: "examples/shard_data.yaml" -database_hosts: - - 'spqr_db_0_1:6432' -database_password: '12345678' -database_port: 6432 -database_max_retries: 5 +cpu_threshold: 1000 +space_threshold: 10000000 +stat_interval_sec: 100000 + +max_move_count: 3 +keys_per_move: 10 diff --git a/examples/coordinator.yaml b/examples/coordinator.yaml index 30e518ae2..259f548ce 100644 --- a/examples/coordinator.yaml +++ b/examples/coordinator.yaml @@ -3,5 +3,5 @@ coordinator_port: 7002 grpc_api_port: 7003 qdb_addr: 'localhost:2379' log_level: info -shard_data: '/spqr/docker/coordinator/shard_data.yaml' +shard_data: 'examples/shard_data.yaml' diff --git a/examples/shard_data.yaml b/examples/shard_data.yaml new file mode 100644 index 000000000..f53a0dad8 --- /dev/null +++ b/examples/shard_data.yaml @@ -0,0 +1,13 @@ +shards: + sh1: + db: db1 + usr: user1 + pwd: 12345678 + hosts: + - 'localhost:5550' + sh2: + db: db1 + usr: user1 + pwd: 12345678 + hosts: + - 'localhost:5551' diff --git a/pkg/clientinteractor/interactor.go b/pkg/clientinteractor/interactor.go index 6c562d244..1bea9493c 100644 --- a/pkg/clientinteractor/interactor.go +++ b/pkg/clientinteractor/interactor.go @@ -308,8 +308,6 @@ func (pi *PSQLInteractor) DropShard(id string) error { // Returns: // - error: An error if sending the messages fails, otherwise nil. func (pi *PSQLInteractor) KeyRanges(krs []*kr.KeyRange) error { - spqrlog.Zero.Debug().Msg("listing key ranges") - for _, msg := range []pgproto3.BackendMessage{ &pgproto3.RowDescription{Fields: []pgproto3.FieldDescription{ TextOidFD("Key range ID"), @@ -331,7 +329,7 @@ func (pi *PSQLInteractor) KeyRanges(krs []*kr.KeyRange) error { []byte(keyRange.ID), []byte(keyRange.ShardID), []byte(keyRange.Distribution), - keyRange.LowerBound, + []byte(strings.Join(keyRange.SendRaw(), ",")), }, }); err != nil { spqrlog.Zero.Error().Err(err).Msg("") @@ -359,7 +357,7 @@ func (pi *PSQLInteractor) CreateKeyRange(ctx context.Context, keyRange *kr.KeyRa } for _, msg := range []pgproto3.BackendMessage{ - &pgproto3.DataRow{Values: [][]byte{[]byte(fmt.Sprintf("bound -> %s", keyRange.LowerBound))}}, + &pgproto3.DataRow{Values: [][]byte{[]byte(fmt.Sprintf("bound -> %s", keyRange.SendRaw()[0]))}}, } { if err := pi.cl.Send(msg); err != nil { spqrlog.Zero.Error().Err(err).Msg("") @@ -389,7 +387,7 @@ func (pi *PSQLInteractor) SplitKeyRange(ctx context.Context, split *kr.SplitKeyR for _, msg := range []pgproto3.BackendMessage{ &pgproto3.DataRow{Values: [][]byte{[]byte(fmt.Sprintf("key range id -> %v", split.SourceID))}}, - &pgproto3.DataRow{Values: [][]byte{[]byte(fmt.Sprintf("bound -> %s", string(split.Bound)))}}, + &pgproto3.DataRow{Values: [][]byte{[]byte(fmt.Sprintf("bound -> %s", string(split.Bound[0])))}}, } { if err := pi.cl.Send(msg); err != nil { spqrlog.Zero.Error().Err(err).Msg("") diff --git a/pkg/coord/adapter.go b/pkg/coord/adapter.go index 32e526543..3db7cef0d 100644 --- a/pkg/coord/adapter.go +++ b/pkg/coord/adapter.go @@ -47,7 +47,7 @@ func NewAdapter(conn *grpc.ClientConn) *Adapter { // Returns: // - qdb.QDB: The QDB object. func (a *Adapter) QDB() qdb.QDB { - return nil + panic("implement me?") } // TODO : unit tests @@ -78,16 +78,26 @@ func (a *Adapter) ShareKeyRange(id string) error { // - error: An error if the retrieval was unsuccessful. func (a *Adapter) GetKeyRange(ctx context.Context, krId string) (*kr.KeyRange, error) { c := proto.NewKeyRangeServiceClient(a.conn) + dc := proto.NewDistributionServiceClient(a.conn) reply, err := c.GetKeyRange(ctx, &proto.GetKeyRangeRequest{ Ids: []string{krId}, }) if err != nil { return nil, err } + // what if len > 1 ? if len(reply.KeyRangesInfo) == 0 { return nil, nil } - return kr.KeyRangeFromProto(reply.KeyRangesInfo[0]), nil + ds, err := dc.GetDistribution(ctx, &proto.GetDistributionRequest{ + Id: reply.KeyRangesInfo[0].DistributionId, + }) + + if err != nil { + return nil, err + } + + return kr.KeyRangeFromProto(reply.KeyRangesInfo[0], ds.Distribution.ColumnTypes), nil } // TODO : unit tests @@ -110,9 +120,15 @@ func (a *Adapter) ListKeyRanges(ctx context.Context, distribution string) ([]*kr return nil, err } + dc := proto.NewDistributionServiceClient(a.conn) + ds, err := dc.GetDistribution(ctx, &proto.GetDistributionRequest{Id: distribution}) + if err != nil { + return nil, err + } + krs := make([]*kr.KeyRange, len(reply.KeyRangesInfo)) for i, keyRange := range reply.KeyRangesInfo { - krs[i] = kr.KeyRangeFromProto(keyRange) + krs[i] = kr.KeyRangeFromProto(keyRange, ds.Distribution.ColumnTypes) } return krs, nil @@ -135,9 +151,15 @@ func (a *Adapter) ListAllKeyRanges(ctx context.Context) ([]*kr.KeyRange, error) return nil, err } + dc := proto.NewDistributionServiceClient(a.conn) + krs := make([]*kr.KeyRange, len(reply.KeyRangesInfo)) for i, keyRange := range reply.KeyRangesInfo { - krs[i] = kr.KeyRangeFromProto(keyRange) + ds, err := dc.GetDistribution(ctx, &proto.GetDistributionRequest{Id: keyRange.DistributionId}) + if err != nil { + return nil, err + } + krs[i] = kr.KeyRangeFromProto(keyRange, ds.Distribution.ColumnTypes) } return krs, nil @@ -236,8 +258,12 @@ func (a *Adapter) Split(ctx context.Context, split *kr.SplitKeyRange) error { for _, keyRange := range krs { if keyRange.ID == split.SourceID { c := proto.NewKeyRangeServiceClient(a.conn) + + nkr := keyRange.ToProto() + nkr.Krid = split.Krid + _, err := c.SplitKeyRange(ctx, &proto.SplitKeyRangeRequest{ - Bound: split.Bound, + Bound: split.Bound[0], // fix multidim case SourceId: split.SourceID, NewId: split.Krid, SplitLeft: split.SplitLeft, @@ -278,7 +304,7 @@ func (a *Adapter) Unite(ctx context.Context, unite *kr.UniteKeyRange) error { } } - if kr.CmpRangesLess(right.LowerBound, left.LowerBound) { + if kr.CmpRangesLess(right.LowerBound, left.LowerBound, right.ColumnTypes) { left, right = right, left } @@ -286,12 +312,12 @@ func (a *Adapter) Unite(ctx context.Context, unite *kr.UniteKeyRange) error { if krCurr.ID == unite.BaseKeyRangeId || krCurr.ID == unite.AppendageKeyRangeId { continue } - if kr.CmpRangesLess(krCurr.LowerBound, right.LowerBound) && kr.CmpRangesLess(left.LowerBound, krCurr.LowerBound) { + if kr.CmpRangesLess(krCurr.LowerBound, right.LowerBound, krCurr.ColumnTypes) && kr.CmpRangesLess(left.LowerBound, krCurr.LowerBound, krCurr.ColumnTypes) { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "unvalid unite request") } } - if left == nil || right == nil || kr.CmpRangesLess(right.LowerBound, left.LowerBound) { + if left == nil || right == nil || kr.CmpRangesLess(right.LowerBound, left.LowerBound, right.ColumnTypes) { return spqrerror.New(spqrerror.SPQR_KEYRANGE_ERROR, "key range on left or right was not found") } diff --git a/pkg/coord/local/clocal.go b/pkg/coord/local/clocal.go index 60c5bb103..d808a9c78 100644 --- a/pkg/coord/local/clocal.go +++ b/pkg/coord/local/clocal.go @@ -34,7 +34,6 @@ type LocalCoordinator struct { qdb qdb.QDB } - // GetTaskGroup retrieves the task group from the local coordinator's QDB. // // Parameters: @@ -113,6 +112,9 @@ func (lc *LocalCoordinator) ListDistributions(ctx context.Context) ([]*distribut func (lc *LocalCoordinator) CreateDistribution(ctx context.Context, ds *distributions.Distribution) error { lc.mu.Lock() defer lc.mu.Unlock() + if len(ds.ColTypes) == 0 { + return fmt.Errorf("empty distributions are disallowed") + } return lc.qdb.CreateDistribution(ctx, distributions.DistributionToDB(ds)) } @@ -131,8 +133,16 @@ func (lc *LocalCoordinator) AlterDistributionAttach(ctx context.Context, id stri lc.mu.Lock() defer lc.mu.Unlock() + ds, err := lc.qdb.GetDistribution(ctx, id) + if err != nil { + return err + } + dRels := []*qdb.DistributedRelation{} for _, r := range rels { + if len(r.DistributionKey) != len(ds.ColTypes) { + return fmt.Errorf("cannot attach relation %v to this dataspace: number of column mismatch", r.Name) + } dRels = append(dRels, distributions.DistributedRelationToDB(r)) } @@ -263,7 +273,6 @@ func (lc *LocalCoordinator) ListShards(ctx context.Context) ([]*datashards.DataS return retShards, nil } - // AddWorldShard adds a world shard to the LocalCoordinator. // // Parameters: @@ -419,6 +428,7 @@ func (lc *LocalCoordinator) WorldShards() []string { return ret } +// Caller should lock key range // TODO : unit tests // Move moves a key range identified by req.Krid to a new shard specified by req.ShardId @@ -436,7 +446,12 @@ func (qr *LocalCoordinator) Move(ctx context.Context, req *kr.MoveKeyRange) erro return err } - var reqKr = kr.KeyRangeFromDB(krmv) + ds, err := qr.qdb.GetDistribution(ctx, krmv.DistributionId) + if err != nil { + return err + } + + var reqKr = kr.KeyRangeFromDB(krmv, ds.ColTypes) reqKr.ShardID = req.ShardId return ops.ModifyKeyRangeWithChecks(ctx, qr.qdb, reqKr) } @@ -459,6 +474,7 @@ func (qr *LocalCoordinator) Unite(ctx context.Context, req *kr.UniteKeyRange) er if krBase, err = qr.qdb.LockKeyRange(ctx, req.BaseKeyRangeId); err != nil { //nolint:all TODO return err } + defer func(qdb qdb.QDB, ctx context.Context, keyRangeID string) { err := qdb.UnlockKeyRange(ctx, keyRangeID) if err != nil { @@ -467,6 +483,11 @@ func (qr *LocalCoordinator) Unite(ctx context.Context, req *kr.UniteKeyRange) er } }(qr.qdb, ctx, req.BaseKeyRangeId) + ds, err := qr.qdb.GetDistribution(ctx, krBase.DistributionId) + if err != nil { + return err + } + // TODO: krRight seems to be empty. if krAppendage, err = qr.qdb.GetKeyRange(ctx, req.AppendageKeyRangeId); err != nil { return err @@ -477,20 +498,18 @@ func (qr *LocalCoordinator) Unite(ctx context.Context, req *kr.UniteKeyRange) er } newBound := krBase.LowerBound - if kr.CmpRangesLess(krAppendage.LowerBound, krBase.LowerBound) { + if kr.CmpRangesLess(kr.KeyRangeFromDB(krAppendage, ds.ColTypes).LowerBound, kr.KeyRangeFromDB(krBase, ds.ColTypes).LowerBound, ds.ColTypes) { newBound = krAppendage.LowerBound } - united := &kr.KeyRange{ - LowerBound: newBound, - ShardID: krBase.ShardID, - Distribution: krBase.DistributionId, - ID: krBase.KeyRangeID, - } + krBaseCopy := krBase + krBaseCopy.LowerBound = newBound + united := kr.KeyRangeFromDB(krBaseCopy, ds.ColTypes) return ops.ModifyKeyRangeWithChecks(ctx, qr.qdb, united) } +// Caller should lock key range // TODO : unit tests // Split splits an existing key range identified by req.SourceID into two new key ranges. @@ -521,28 +540,37 @@ func (qr *LocalCoordinator) Split(ctx context.Context, req *kr.SplitKeyRange) er } }() - krNew := &kr.KeyRange{ - LowerBound: func() []byte { - if req.SplitLeft { - return krOld.LowerBound - } - return req.Bound - }(), - ID: req.Krid, - ShardID: krOld.ShardID, - Distribution: krOld.DistributionId, + ds, err := qr.qdb.GetDistribution(ctx, krOld.DistributionId) + if err != nil { + return err } + krNew := kr.KeyRangeFromDB( + &qdb.KeyRange{ + LowerBound: func() [][]byte { + if req.SplitLeft { + return krOld.LowerBound + } + return req.Bound // fix multidim case ! + }(), + KeyRangeID: req.Krid, + ShardID: krOld.ShardID, + DistributionId: krOld.DistributionId, + }, + ds.ColTypes, + ) + spqrlog.Zero.Debug(). - Bytes("lower-bound", krNew.LowerBound). + Bytes("lower-bound", krNew.Raw()[0]). Str("shard-id", krNew.ShardID). Str("id", krNew.ID). Msg("new key range") if req.SplitLeft { - krOld.LowerBound = req.Bound + krOld.LowerBound = req.Bound // TODO: fix } - if err := ops.ModifyKeyRangeWithChecks(ctx, qr.qdb, kr.KeyRangeFromDB(krOld)); err != nil { + + if err := ops.ModifyKeyRangeWithChecks(ctx, qr.qdb, kr.KeyRangeFromDB(krOld, ds.ColTypes)); err != nil { return err } @@ -570,7 +598,12 @@ func (qr *LocalCoordinator) LockKeyRange(ctx context.Context, krid string) (*kr. return nil, err } - return kr.KeyRangeFromDB(keyRangeDB), nil + ds, err := qr.qdb.GetDistribution(ctx, keyRangeDB.DistributionId) + if err != nil { + return nil, err + } + + return kr.KeyRangeFromDB(keyRangeDB, ds.ColTypes), nil } // TODO : unit tests @@ -646,7 +679,11 @@ func (lc *LocalCoordinator) GetKeyRange(ctx context.Context, krId string) (*kr.K if err != nil { return nil, err } - return kr.KeyRangeFromDB(krDb), nil + ds, err := lc.qdb.GetDistribution(ctx, krDb.DistributionId) + if err != nil { + return nil, err + } + return kr.KeyRangeFromDB(krDb, ds.ColTypes), nil } // TODO : unit tests @@ -666,8 +703,13 @@ func (qr *LocalCoordinator) ListKeyRanges(ctx context.Context, distribution stri return nil, err } else { for _, keyRange := range krs { - ret = append(ret, kr.KeyRangeFromDB(keyRange)) + ds, err := qr.qdb.GetDistribution(ctx, keyRange.DistributionId) + + if err != nil { + return nil, err + } + ret = append(ret, kr.KeyRangeFromDB(keyRange, ds.ColTypes)) } } @@ -685,17 +727,28 @@ func (qr *LocalCoordinator) ListKeyRanges(ctx context.Context, distribution stri // - []*kr.KeyRange: a slice of KeyRange objects representing all key ranges. // - error: an error if the retrieval encounters any issues. func (qr *LocalCoordinator) ListAllKeyRanges(ctx context.Context) ([]*kr.KeyRange, error) { - var ret []*kr.KeyRange if krs, err := qr.qdb.ListAllKeyRanges(ctx); err != nil { return nil, err } else { + var ret []*kr.KeyRange + cache := map[string]*qdb.Distribution{} + for _, keyRange := range krs { - ret = append(ret, kr.KeyRangeFromDB(keyRange)) + var ds *qdb.Distribution + var err error + var ok bool + if ds, ok = cache[keyRange.DistributionId]; !ok { + ds, err = qr.qdb.GetDistribution(ctx, keyRange.DistributionId) + if err != nil { + return nil, err + } + cache[keyRange.DistributionId] = ds + } + ret = append(ret, kr.KeyRangeFromDB(keyRange, ds.ColTypes)) } + return ret, nil } - - return ret, nil } // TODO : unit tests @@ -800,7 +853,6 @@ func (qr *LocalCoordinator) UpdateCoordinator(ctx context.Context, addr string) return qr.qdb.UpdateCoordinator(ctx, addr) } - // GetCoordinator retrieves the coordinator address from the local coordinator. // // Parameters: diff --git a/pkg/coord/local/clocal_test.go b/pkg/coord/local/clocal_test.go new file mode 100644 index 000000000..1db541a3b --- /dev/null +++ b/pkg/coord/local/clocal_test.go @@ -0,0 +1,82 @@ +package local_test + +import ( + "context" + "testing" + + "github.com/golang/mock/gomock" + "github.com/pg-sharding/spqr/pkg/coord/local" + "github.com/pg-sharding/spqr/qdb" + mock "github.com/pg-sharding/spqr/qdb/mock" + "github.com/stretchr/testify/assert" +) + +func TestListKeyRangesCaches(t *testing.T) { + + assert := assert.New(t) + ctrl := gomock.NewController(t) + + db := mock.NewMockQDB(ctrl) + + lc := local.NewLocalCoordinator(db) + + krs := []*qdb.KeyRange{ + { + LowerBound: [][]byte{[]byte("1")}, + ShardID: "sh1", + KeyRangeID: "kr1", + DistributionId: "ds1", + }, + + { + LowerBound: [][]byte{[]byte("2")}, + ShardID: "sh2", + KeyRangeID: "kr2", + DistributionId: "ds1", + }, + + { + LowerBound: [][]byte{[]byte("3")}, + ShardID: "sh3", + KeyRangeID: "kr3", + DistributionId: "ds1", + }, + + { + LowerBound: [][]byte{[]byte("4")}, + ShardID: "sh4", + KeyRangeID: "kr4", + DistributionId: "ds2", + }, + + { + LowerBound: [][]byte{[]byte("5")}, + ShardID: "sh5", + KeyRangeID: "kr5", + DistributionId: "ds2", + }, + } + db.EXPECT().ListAllKeyRanges(gomock.All()).Return(krs, nil) + /* check that request caches */ + + ds1 := &qdb.Distribution{ + ID: "ds1", + ColTypes: []string{ + qdb.ColumnTypeVarchar, + }, + } + + ds2 := &qdb.Distribution{ + ID: "ds2", + ColTypes: []string{ + qdb.ColumnTypeVarchar, + }, + } + + db.EXPECT().GetDistribution(gomock.Any(), "ds1").Times(1).Return(ds1, nil) + db.EXPECT().GetDistribution(gomock.Any(), "ds2").Times(1).Return(ds2, nil) + + _, err := lc.ListAllKeyRanges(context.Background()) + + assert.NoError(err) +} diff --git a/pkg/datatransfers/data_transfers.go b/pkg/datatransfers/data_transfers.go index 31005741f..0dca0ae7a 100644 --- a/pkg/datatransfers/data_transfers.go +++ b/pkg/datatransfers/data_transfers.go @@ -3,18 +3,20 @@ package datatransfers import ( "context" "fmt" - "github.com/jackc/pgx/v5" + "io" + "os" + "strings" + "sync" + + "github.com/pg-sharding/spqr/pkg/models/distributions" + + pgx "github.com/jackc/pgx/v5" _ "github.com/lib/pq" "github.com/pg-sharding/spqr/coordinator" "github.com/pg-sharding/spqr/pkg/config" - "github.com/pg-sharding/spqr/pkg/models/distributions" "github.com/pg-sharding/spqr/pkg/models/kr" "github.com/pg-sharding/spqr/pkg/spqrlog" "github.com/pg-sharding/spqr/qdb" - "io" - "os" - "strings" - "sync" ) type MoveTableRes struct { @@ -107,9 +109,9 @@ Steps: // // It is assumed that the passed key range is already locked on every online spqr-router. // The function performs the following steps: -// - Create a postgres_fdw on the receiving shard. -// - Copy data from the sending shard to the receiving shard via fdw. -// - Delete data from the sending shard. +// - Create a postgres_fdw on the receiving shard. +// - Copy data from the sending shard to the receiving shard via fdw. +// - Delete data from the sending shard. // // Parameters: // - ctx (context.Context): The context for the function. @@ -202,7 +204,6 @@ func MoveKeys(ctx context.Context, fromId, toId string, krg *kr.KeyRange, ds *di return nil } - // resolveNextBound finds the next lower bound key range from the given key range list that is greater than the lower bound of the given key range. // // Parameters: @@ -218,9 +219,13 @@ func resolveNextBound(ctx context.Context, krg *kr.KeyRange, cr coordinator.Coor if err != nil { return nil, err } + ds, err := cr.GetDistribution(ctx, krg.Distribution) + if err != nil { + return nil, err + } var bound kr.KeyRangeBound for _, kRange := range krs { - if kr.CmpRangesLess(krg.LowerBound, kRange.LowerBound) && (bound == nil || kr.CmpRangesLess(kRange.LowerBound, bound)) { + if kr.CmpRangesLess(krg.LowerBound, kRange.LowerBound, ds.ColTypes) && (bound == nil || kr.CmpRangesLess(kRange.LowerBound, bound, ds.ColTypes)) { bound = kRange.LowerBound } } @@ -231,8 +236,8 @@ func resolveNextBound(ctx context.Context, krg *kr.KeyRange, cr coordinator.Coor // // It is assumed that the passed key range is already locked on every online spqr-router. // The function performs the following steps: -// - Create a postgres_fdw on the receiving shard. -// - Copy data from the sending shard to the receiving shard via fdw. +// - Create a postgres_fdw on the receiving shard. +// - Copy data from the sending shard to the receiving shard via fdw. // // Parameters: // - ctx (context.Context): The context for the function. diff --git a/pkg/decode/spqrql.go b/pkg/decode/spqrql.go index 9de8f0adc..4a71feec3 100644 --- a/pkg/decode/spqrql.go +++ b/pkg/decode/spqrql.go @@ -4,6 +4,7 @@ import ( "fmt" "strings" + "github.com/pg-sharding/spqr/pkg/models/kr" protos "github.com/pg-sharding/spqr/pkg/protos" ) @@ -15,9 +16,9 @@ import ( // // Returns: // - string: the SQL query to create the key range. -func KeyRange(krg *protos.KeyRangeInfo) string { +func KeyRange(krg *kr.KeyRange) string { /* TODO: composite key support */ - return fmt.Sprintf("CREATE KEY RANGE %s FROM %s ROUTE TO %s FOR DISTRIBUTION %s;", krg.Krid, krg.KeyRange.LowerBound, krg.ShardId, krg.DistributionId) + return fmt.Sprintf("CREATE KEY RANGE %s FROM %s ROUTE TO %s FOR DISTRIBUTION %s;", krg.ID, strings.Join(krg.SendRaw(), ","), krg.ShardID, krg.Distribution) } // Distribution returns query to create given distribution diff --git a/pkg/decode/spqrql_test.go b/pkg/decode/spqrql_test.go index d2d532d0a..718b0d8c2 100644 --- a/pkg/decode/spqrql_test.go +++ b/pkg/decode/spqrql_test.go @@ -1,9 +1,12 @@ package decode import ( + "testing" + + "github.com/pg-sharding/spqr/pkg/models/kr" protos "github.com/pg-sharding/spqr/pkg/protos" + "github.com/pg-sharding/spqr/qdb" "github.com/stretchr/testify/assert" - "testing" ) @@ -20,14 +23,25 @@ import ( func TestKeyRange(t *testing.T) { assert := assert.New(t) assert.Equal("CREATE KEY RANGE kr1 FROM 10 ROUTE TO sh1 FOR DISTRIBUTION ds1;", - KeyRange(&protos.KeyRangeInfo{ - Krid: "kr1", - ShardId: "sh1", - DistributionId: "ds1", - KeyRange: &protos.KeyRange{ - LowerBound: "10", - }, + KeyRange(&kr.KeyRange{ + ID: "kr1", + ShardID: "sh1", + Distribution: "ds1", + LowerBound: []interface{}{10}, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, })) + // UpperBound is ignored + assert.Equal("CREATE KEY RANGE kr1 FROM 10 ROUTE TO sh1 FOR DISTRIBUTION ds1;", + KeyRange( + &kr.KeyRange{ + ID: "kr1", + ShardID: "sh1", + Distribution: "ds1", + LowerBound: []interface{}{10}, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + })) } // TestDistribution is a unit test function for the Distribution function. diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index 4951f36d3..f10cf5c0f 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -192,7 +192,16 @@ func processCreate(ctx context.Context, astmt spqrparser.Statement, mngr EntityM case *spqrparser.ShardingRuleDefinition: return cli.ReportError(spqrerror.ShardingRulesRemoved) case *spqrparser.KeyRangeDefinition: - req := kr.KeyRangeFromSQL(stmt) + ds, err := mngr.GetDistribution(ctx, stmt.Distribution) + if err != nil { + spqrlog.Zero.Error().Err(err).Msg("Error when adding key range") + return cli.ReportError(err) + } + req, err := kr.KeyRangeFromSQL(stmt, ds.ColTypes) + if err != nil { + spqrlog.Zero.Error().Err(err).Msg("Error when adding key range") + return cli.ReportError(err) + } if err := mngr.CreateKeyRange(ctx, req); err != nil { spqrlog.Zero.Error().Err(err).Msg("Error when adding key range") return cli.ReportError(err) @@ -347,7 +356,7 @@ func Proc(ctx context.Context, tstmt spqrparser.Statement, mgr EntityMgr, ci con return ProcessKill(ctx, stmt, mgr, ci, cli) case *spqrparser.SplitKeyRange: splitKeyRange := &kr.SplitKeyRange{ - Bound: stmt.Border, + Bound: stmt.Border.Pivots, SourceID: stmt.KeyRangeFromID, Krid: stmt.KeyRangeID, } diff --git a/pkg/models/hashfunction/hashfunction.go b/pkg/models/hashfunction/hashfunction.go index 7059233f4..305029720 100644 --- a/pkg/models/hashfunction/hashfunction.go +++ b/pkg/models/hashfunction/hashfunction.go @@ -1,10 +1,11 @@ package hashfunction import ( + "encoding/binary" "fmt" - "strconv" "github.com/go-faster/city" + "github.com/pg-sharding/spqr/qdb" "github.com/spaolacci/murmur3" ) @@ -29,18 +30,70 @@ var ( // - hf: The hash function to apply. // // Returns: -// - []byte: The hashed byte slice. +// - []interface: The hashed byte slice. // - error: An error if any error occurs during the process. -func ApplyHashFunction(inp []byte, hf HashFunctionType) ([]byte, error) { +func ApplyHashFunction(inp interface{}, ctype string, hf HashFunctionType) (interface{}, error) { switch hf { case HashFunctionIdent: return inp, nil case HashFunctionMurmur: - h := murmur3.Sum32(inp) - return []byte(strconv.FormatUint(uint64(h), 10)), nil + switch ctype { + case qdb.ColumnTypeInteger: + buf := make([]byte, 8) + binary.PutVarint(buf, inp.(int64)) + h := murmur3.Sum32(buf) + return uint64(h), nil + + case qdb.ColumnTypeUinteger: + buf := make([]byte, 8) + binary.PutUvarint(buf, inp.(uint64)) + h := murmur3.Sum32(buf) + return uint64(h), nil + case qdb.ColumnTypeVarcharHashed: + switch v := inp.(type) { + case []byte: + h := murmur3.Sum32(v) + + return uint64(h), nil + case string: + h := murmur3.Sum32([]byte(v)) + + return uint64(h), nil + default: + return nil, errNoSuchHashFunction + } + default: + return nil, errNoSuchHashFunction + } case HashFunctionCity: - h := city.Hash32(inp) - return []byte(strconv.FormatUint(uint64(h), 10)), nil + switch ctype { + case qdb.ColumnTypeInteger: + buf := make([]byte, 8) + binary.PutVarint(buf, inp.(int64)) + h := city.Hash32(buf) + return uint64(h), nil + + case qdb.ColumnTypeUinteger: + buf := make([]byte, 8) + binary.PutUvarint(buf, inp.(uint64)) + h := city.Hash32(buf) + return uint64(h), nil + case qdb.ColumnTypeVarcharHashed: + switch v := inp.(type) { + case []byte: + h := city.Hash32(v) + + return uint64(h), nil + case string: + h := city.Hash32([]byte(v)) + + return uint64(h), nil + default: + return nil, errNoSuchHashFunction + } + default: + return nil, errNoSuchHashFunction + } default: return nil, errNoSuchHashFunction } diff --git a/pkg/models/kr/keyrange.go b/pkg/models/kr/keyrange.go index 279521c0d..10a9bf7a9 100644 --- a/pkg/models/kr/keyrange.go +++ b/pkg/models/kr/keyrange.go @@ -1,6 +1,7 @@ package kr import ( + "encoding/binary" "fmt" "strings" @@ -10,78 +11,232 @@ import ( spqrparser "github.com/pg-sharding/spqr/yacc/console" ) -type KeyRangeBound []byte +type KeyRangeBound []interface{} type ShardKey struct { Name string RW bool } +// qdb KeyRange with its distibtion column types +// stored in case struct for fast convertion/access type KeyRange struct { LowerBound KeyRangeBound ShardID string ID string Distribution string + + ColumnTypes []string } -// CmpRangesLess compares two byte slices, kr and other, and returns true if kr is less than other. -// The comparison is based on the length of the slices and the lexicographic order of their string representations. -// -// Parameters: -// - kr: The first byte slice to compare. -// - other: The second byte slice to compare. -// -// Returns: -// - bool: True if kr is less than other, false otherwise. -// +/* +* Old style key ranges comparation + */ // TODO : unit tests -func CmpRangesLess(kr []byte, other []byte) bool { - if len(kr) == len(other) { - return string(kr) < string(other) +func CmpRangesLessStringsDeprecated(bound string, key string) bool { + if len(bound) == len(key) { + return bound < key } - return len(kr) < len(other) + return len(bound) < len(key) } -// CmpRangesLessEqual compares two byte slices, kr and other, and returns true if kr is less than or equal to other. -// The comparison is done by comparing the lengths of the slices first. If the lengths are equal, the function compares the byte values lexicographically. -// Returns true if kr is less than or equal to other, false otherwise. -// -// Parameters: -// - kr: The first byte slice to compare. -// - other: The second byte slice to compare. -// -// Returns: -// - bool: True if kr is less than or equal to other, false otherwise. -// -// TODO : unit tests -func CmpRangesLessEqual(kr []byte, other []byte) bool { - if len(kr) == len(other) { - return string(kr) <= string(other) +func (kr *KeyRange) InFunc(attribInd int, raw []byte) { + switch kr.ColumnTypes[attribInd] { + case qdb.ColumnTypeInteger: + n, _ := binary.Varint(raw) + kr.LowerBound[attribInd] = n + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeUinteger: + /* TODO: fix */ + n, _ := binary.Varint(raw) + kr.LowerBound[attribInd] = uint64(n) + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarchar: + kr.LowerBound[attribInd] = string(raw) + } +} + +func (kr *KeyRange) OutFunc(attribInd int) []byte { + switch kr.ColumnTypes[attribInd] { + case qdb.ColumnTypeInteger: + raw := make([]byte, 8) + _ = binary.PutVarint(raw, kr.LowerBound[attribInd].(int64)) + return raw + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeUinteger: + raw := make([]byte, 8) + _ = binary.PutUvarint(raw, kr.LowerBound[attribInd].(uint64)) + return raw + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarchar: + return []byte(kr.LowerBound[attribInd].(string)) + } + return nil +} + +func (kr *KeyRange) SendFunc(attribInd int) string { + switch kr.ColumnTypes[attribInd] { + case qdb.ColumnTypeInteger: + fallthrough + /* Is uint */ + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeUinteger: + return fmt.Sprintf("%v", kr.LowerBound[attribInd]) + default: + return fmt.Sprintf("'%v'", kr.LowerBound[attribInd]) + } +} + +func (kr *KeyRange) Raw() [][]byte { + res := make([][]byte, len(kr.ColumnTypes)) + + for i := 0; i < len(kr.ColumnTypes); i++ { + res[i] = kr.OutFunc(i) } - return len(kr) < len(other) + return res } -// CmpRangesEqual compares two byte slices, kr and other, and returns true if they are equal. -// It checks if the lengths of kr and other are the same, and then compares their string representations. +func (kr *KeyRange) SendRaw() []string { + res := make([]string, len(kr.ColumnTypes)) + + for i := 0; i < len(kr.ColumnTypes); i++ { + res[i] = kr.SendFunc(i) + } + + return res +} + +// TODO: use it +var MissTypedKeyRange = fmt.Errorf("key range bound is mistyped") + +// CmpRangesLess compares two byte slices, kr and other, and returns true if kr is less than other. +// The comparison is based on the length of the slices and the lexicographic order of their string representations. // // Parameters: // - kr: The first byte slice to compare. // - other: The second byte slice to compare. // // Returns: -// - bool: True if kr and other are equal, false otherwise. +// - bool: True if kr is less than other, false otherwise. // // TODO : unit tests -func CmpRangesEqual(kr []byte, other []byte) bool { - if len(kr) == len(other) { - return string(kr) == string(other) +func CmpRangesLess(bound KeyRangeBound, key KeyRangeBound, types []string) bool { + // Here we panic if we failed to convert key range bound + // element to expected type. We consider panic as much better + // result that data corruption caused by erroreus routing logic. + // Big TODO here is to use and check specific error of types mismatch. + + for i := 0; i < len(bound); i++ { + switch types[i] { + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeUinteger: + i1 := bound[i].(uint64) + i2 := key[i].(uint64) + if i1 == i2 { + // continue + } else if i1 < i2 { + return true + } else { + return false + } + case qdb.ColumnTypeInteger: + i1 := bound[i].(int64) + i2 := key[i].(int64) + if i1 == i2 { + // continue + } else if i1 < i2 { + return true + } else { + return false + } + case qdb.ColumnTypeVarchar: + i1 := bound[i].(string) + i2 := key[i].(string) + if i1 == i2 { + // continue + } else if i1 < i2 { + return true + } else { + return false + } + case qdb.ColumnTypeVarcharDeprecated: + i1 := bound[i].(string) + i2 := key[i].(string) + if i1 == i2 { + // continue + } else if CmpRangesLessStringsDeprecated(i1, i2) { + return true + } else { + return false + } + default: + panic(MissTypedKeyRange) + } } + // keys are actually equal. return false return false } +func CmpRangesEqual(bound KeyRangeBound, key KeyRangeBound, types []string) bool { + for i := 0; i < len(bound); i++ { + switch types[i] { + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeUinteger: + i1 := bound[i].(uint64) + i2 := key[i].(uint64) + if i1 == i2 { + // continue + } else { + return false + } + case qdb.ColumnTypeInteger: + i1 := bound[i].(int64) + i2 := key[i].(int64) + if i1 == i2 { + // continue + } else { + return false + } + case qdb.ColumnTypeVarchar: + i1 := bound[i].(string) + i2 := key[i].(string) + if i1 == i2 { + // continue + + } else { + return false + } + case qdb.ColumnTypeVarcharDeprecated: + i1 := bound[i].(string) + i2 := key[i].(string) + if i1 == i2 { + // continue + } else { + return false + } + default: + panic(MissTypedKeyRange) + } + } + + // keys are actually equal. + return true +} + +func CmpRangesLessEqual(bound KeyRangeBound, key KeyRangeBound, types []string) bool { + return CmpRangesEqual(bound, key, types) || CmpRangesLess(bound, key, types) +} + // KeyRangeFromDB converts a qdb.KeyRange object to a KeyRange object. // It creates a new KeyRange object with the values from the qdb.KeyRange object. // It returns a pointer to the new KeyRange object. @@ -93,13 +248,23 @@ func CmpRangesEqual(kr []byte, other []byte) bool { // - *KeyRange: A pointer to the new KeyRange object. // // TODO : unit tests -func KeyRangeFromDB(kr *qdb.KeyRange) *KeyRange { - return &KeyRange{ - LowerBound: kr.LowerBound, - ShardID: kr.ShardID, - ID: kr.KeyRangeID, - Distribution: kr.DistributionId, +func KeyRangeFromDB(krdb *qdb.KeyRange, colTypes []string) *KeyRange { + kr := &KeyRange{ + ShardID: krdb.ShardID, + ID: krdb.KeyRangeID, + Distribution: krdb.DistributionId, + ColumnTypes: colTypes, + + LowerBound: make(KeyRangeBound, len(colTypes)), + } + + // TODO: Fix this! (krdb.LowerBound -> krqb.LowerBound[i]) + // now this works only for unidim distributions + for i := 0; i < len(colTypes); i++ { + kr.InFunc(i, krdb.LowerBound[i]) } + + return kr } // KeyRangeFromSQL converts a spqrparser.KeyRangeDefinition into a KeyRange. @@ -113,16 +278,44 @@ func KeyRangeFromDB(kr *qdb.KeyRange) *KeyRange { // - *KeyRange: A pointer to the new KeyRange object. // // TODO : unit tests -func KeyRangeFromSQL(kr *spqrparser.KeyRangeDefinition) *KeyRange { - if kr == nil { - return nil +func KeyRangeFromSQL(krsql *spqrparser.KeyRangeDefinition, colTypes []string) (*KeyRange, error) { + if krsql == nil { + return nil, nil } - return &KeyRange{ - LowerBound: kr.LowerBound, - ShardID: kr.ShardID, - ID: kr.KeyRangeID, - Distribution: kr.Distribution, + kr := &KeyRange{ + ShardID: krsql.ShardID, + ID: krsql.KeyRangeID, + Distribution: krsql.Distribution, + + ColumnTypes: colTypes, + + LowerBound: make(KeyRangeBound, len(colTypes)), + } + + if len(colTypes) != len(krsql.LowerBound.Pivots) { + return nil, fmt.Errorf("number of columns mismatches with distribution") + } + + for i := 0; i < len(colTypes); i++ { + kr.InFunc(i, krsql.LowerBound.Pivots[i]) } + + return kr, nil +} + +func KeyRangeFromBytes(val [][]byte, colTypes []string) *KeyRange { + + kr := &KeyRange{ + ColumnTypes: colTypes, + + LowerBound: make(KeyRangeBound, len(colTypes)), + } + + for i := 0; i < len(colTypes); i++ { + kr.InFunc(i, val[i]) + } + + return kr } // KeyRangeFromProto converts a protobuf KeyRangeInfo to a KeyRange object. @@ -131,21 +324,33 @@ func KeyRangeFromSQL(kr *spqrparser.KeyRangeDefinition) *KeyRange { // // Parameters: // - kr: The protobuf KeyRangeInfo to convert. +// - colTypes: the column types list // // Returns: // - *KeyRange: A pointer to the new KeyRange object. // // TODO : unit tests -func KeyRangeFromProto(kr *proto.KeyRangeInfo) *KeyRange { - if kr == nil { +func KeyRangeFromProto(krproto *proto.KeyRangeInfo, colTypes []string) *KeyRange { + if krproto == nil { return nil } - return &KeyRange{ - LowerBound: KeyRangeBound(kr.KeyRange.LowerBound), - ShardID: kr.ShardId, - ID: kr.Krid, - Distribution: kr.DistributionId, + kr := &KeyRange{ + ShardID: krproto.ShardId, + ID: krproto.Krid, + Distribution: krproto.DistributionId, + ColumnTypes: colTypes, + + LowerBound: make(KeyRangeBound, len(colTypes)), + } + //if len(colTypes) != len(krsql.LowerBound.Pivots) { + // return nil, fmt.Errorf("number of columns mismatches with distribution") + //} + + for i := 0; i < len(colTypes); i++ { + kr.InFunc(i, krproto.Bound.Values[i]) } + + return kr } // ToDB converts the KeyRange struct to a qdb.KeyRange struct. @@ -156,12 +361,16 @@ func KeyRangeFromProto(kr *proto.KeyRangeInfo) *KeyRange { // // TODO : unit tests func (kr *KeyRange) ToDB() *qdb.KeyRange { - return &qdb.KeyRange{ - LowerBound: kr.LowerBound, + krDb := &qdb.KeyRange{ + LowerBound: make([][]byte, len(kr.ColumnTypes)), ShardID: kr.ShardID, KeyRangeID: kr.ID, DistributionId: kr.Distribution, } + for i := 0; i < len(kr.ColumnTypes); i++ { + krDb.LowerBound[i] = kr.OutFunc(i) + } + return krDb } // ToProto converts the KeyRange struct to a protobuf KeyRangeInfo message. @@ -172,14 +381,20 @@ func (kr *KeyRange) ToDB() *qdb.KeyRange { // // TODO : unit tests func (kr *KeyRange) ToProto() *proto.KeyRangeInfo { - return &proto.KeyRangeInfo{ - KeyRange: &proto.KeyRange{ - LowerBound: string(kr.LowerBound), + krProto := &proto.KeyRangeInfo{ + Bound: &proto.KeyRangeBound{ + Values: make([][]byte, len(kr.ColumnTypes)), }, ShardId: kr.ShardID, Krid: kr.ID, DistributionId: kr.Distribution, } + + for i := 0; i < len(kr.ColumnTypes); i++ { + krProto.Bound.Values[i] = kr.OutFunc(i) + } + + return krProto } // GetKRCondition returns SQL condition for elements of distributed relation between two key ranges @@ -201,29 +416,25 @@ func GetKRCondition(ds *distributions.Distribution, rel *distributions.Distribut if i > 0 { break } + // TODO add hash (depends on col type) - hashedCol := "" + fqCol := "" if prefix != "" { - hashedCol = fmt.Sprintf("%s.%s", prefix, entry.Column) + fqCol = fmt.Sprintf("%s.%s", prefix, entry.Column) } else { - hashedCol = entry.Column + fqCol = entry.Column } - lBound := "" - if ds.ColTypes[i] == "varchar" { - lBound = fmt.Sprintf("'%s'", string(kRange.LowerBound)) - } else { - lBound = string(kRange.LowerBound) + + krTmp := KeyRange{ + + LowerBound: upperBound, + ColumnTypes: kRange.ColumnTypes, } + if upperBound != nil { - rBound := "" - if ds.ColTypes[i] == "varchar" { - rBound = fmt.Sprintf("'%s'", string(upperBound)) - } else { - rBound = string(upperBound) - } - buf[i] = fmt.Sprintf("%s >= %s AND %s < %s", hashedCol, lBound, hashedCol, rBound) + buf[i] = fmt.Sprintf("%s >= %s AND %s < %s", fqCol, kRange.SendFunc(i), fqCol, krTmp.SendFunc(i)) } else { - buf[i] = fmt.Sprintf("%s >= %s", hashedCol, lBound) + buf[i] = fmt.Sprintf("%s >= %s", fqCol, kRange.SendFunc(i)) } } return strings.Join(buf, " AND ") diff --git a/pkg/models/kr/keyrange_test.go b/pkg/models/kr/keyrange_test.go index 817d46485..3aa86aaa5 100644 --- a/pkg/models/kr/keyrange_test.go +++ b/pkg/models/kr/keyrange_test.go @@ -31,8 +31,8 @@ func TestGetKRCondition(t *testing.T) { {Column: "col1", HashFunction: "ident"}, }, }, - krg: &kr.KeyRange{ID: "kr1", LowerBound: []byte("0")}, - upperBound: []byte("10"), + krg: &kr.KeyRange{ID: "kr1", LowerBound: []interface{}{0}, ColumnTypes: []string{"integer"}}, + upperBound: []interface{}{10}, prefix: "", expected: "col1 >= 0 AND col1 < 10", }, @@ -45,8 +45,13 @@ func TestGetKRCondition(t *testing.T) { {Column: "col1", HashFunction: "ident"}, }, }, - krg: &kr.KeyRange{ID: "kr1", LowerBound: []byte("0")}, - upperBound: []byte("10"), + krg: &kr.KeyRange{ID: "kr1", LowerBound: []interface { + }{ + 0, + }, + ColumnTypes: []string{"integer"}, + }, + upperBound: []interface{}{10}, prefix: "rel", expected: "rel.col1 >= 0 AND rel.col1 < 10", }, @@ -59,7 +64,7 @@ func TestGetKRCondition(t *testing.T) { {Column: "col1", HashFunction: "ident"}, }, }, - krg: &kr.KeyRange{ID: "kr1", LowerBound: []byte("0")}, + krg: &kr.KeyRange{ID: "kr1", LowerBound: []interface{}{0}, ColumnTypes: []string{"integer"}}, upperBound: nil, prefix: "", expected: "col1 >= 0", @@ -73,8 +78,8 @@ func TestGetKRCondition(t *testing.T) { {Column: "col1", HashFunction: "ident"}, }, }, - krg: &kr.KeyRange{ID: "kr1", LowerBound: []byte("a")}, - upperBound: []byte("b"), + krg: &kr.KeyRange{ID: "kr1", LowerBound: []interface{}{"a"}, ColumnTypes: []string{"varchar"}}, + upperBound: []interface{}{"b"}, prefix: "", expected: "col1 >= 'a' AND col1 < 'b'", }, @@ -85,5 +90,4 @@ func TestGetKRCondition(t *testing.T) { "test case %d", i, ) } - } diff --git a/pkg/models/kr/keyrangemgr.go b/pkg/models/kr/keyrangemgr.go index 4a4c5c644..8e1b2abe9 100644 --- a/pkg/models/kr/keyrangemgr.go +++ b/pkg/models/kr/keyrangemgr.go @@ -3,7 +3,7 @@ package kr import "context" type SplitKeyRange struct { - Bound KeyRangeBound + Bound [][]byte // KeyRangeBound raw SourceID string Krid string SplitLeft bool diff --git a/pkg/protos/key_range.pb.go b/pkg/protos/key_range.pb.go index b2a035fc5..9756d9fcf 100644 --- a/pkg/protos/key_range.pb.go +++ b/pkg/protos/key_range.pb.go @@ -66,16 +66,16 @@ func (KeyRangeStatus) EnumDescriptor() ([]byte, []int) { return file_protos_key_range_proto_rawDescGZIP(), []int{0} } -type KeyRange struct { +type KeyRangeBound struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LowerBound string `protobuf:"bytes,1,opt,name=lower_bound,json=lowerBound,proto3" json:"lower_bound,omitempty"` + Values [][]byte `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` } -func (x *KeyRange) Reset() { - *x = KeyRange{} +func (x *KeyRangeBound) Reset() { + *x = KeyRangeBound{} if protoimpl.UnsafeEnabled { mi := &file_protos_key_range_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -83,13 +83,13 @@ func (x *KeyRange) Reset() { } } -func (x *KeyRange) String() string { +func (x *KeyRangeBound) String() string { return protoimpl.X.MessageStringOf(x) } -func (*KeyRange) ProtoMessage() {} +func (*KeyRangeBound) ProtoMessage() {} -func (x *KeyRange) ProtoReflect() protoreflect.Message { +func (x *KeyRangeBound) ProtoReflect() protoreflect.Message { mi := &file_protos_key_range_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -101,16 +101,16 @@ func (x *KeyRange) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use KeyRange.ProtoReflect.Descriptor instead. -func (*KeyRange) Descriptor() ([]byte, []int) { +// Deprecated: Use KeyRangeBound.ProtoReflect.Descriptor instead. +func (*KeyRangeBound) Descriptor() ([]byte, []int) { return file_protos_key_range_proto_rawDescGZIP(), []int{0} } -func (x *KeyRange) GetLowerBound() string { +func (x *KeyRangeBound) GetValues() [][]byte { if x != nil { - return x.LowerBound + return x.Values } - return "" + return nil } // key range info is mapped to shard @@ -119,10 +119,10 @@ type KeyRangeInfo struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - KeyRange *KeyRange `protobuf:"bytes,1,opt,name=key_range,json=keyRange,proto3" json:"key_range,omitempty"` - Krid string `protobuf:"bytes,2,opt,name=krid,proto3" json:"krid,omitempty"` - ShardId string `protobuf:"bytes,3,opt,name=shardId,proto3" json:"shardId,omitempty"` - DistributionId string `protobuf:"bytes,4,opt,name=distributionId,proto3" json:"distributionId,omitempty"` + Bound *KeyRangeBound `protobuf:"bytes,1,opt,name=bound,proto3" json:"bound,omitempty"` + Krid string `protobuf:"bytes,2,opt,name=krid,proto3" json:"krid,omitempty"` + ShardId string `protobuf:"bytes,3,opt,name=shardId,proto3" json:"shardId,omitempty"` + DistributionId string `protobuf:"bytes,4,opt,name=distributionId,proto3" json:"distributionId,omitempty"` } func (x *KeyRangeInfo) Reset() { @@ -157,9 +157,9 @@ func (*KeyRangeInfo) Descriptor() ([]byte, []int) { return file_protos_key_range_proto_rawDescGZIP(), []int{1} } -func (x *KeyRangeInfo) GetKeyRange() *KeyRange { +func (x *KeyRangeInfo) GetBound() *KeyRangeBound { if x != nil { - return x.KeyRange + return x.Bound } return nil } @@ -963,136 +963,135 @@ var File_protos_key_range_proto protoreflect.FileDescriptor var file_protos_key_range_proto_rawDesc = []byte{ 0x0a, 0x16, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, - 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x73, 0x70, 0x71, 0x72, 0x22, 0x2b, - 0x0a, 0x08, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, - 0x77, 0x65, 0x72, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x22, 0x91, 0x01, 0x0a, 0x0c, - 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2b, 0x0a, 0x09, - 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0e, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, - 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x72, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6b, 0x72, 0x69, 0x64, 0x12, 0x18, 0x0a, - 0x07, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x64, 0x69, 0x73, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0e, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, - 0x39, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, - 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x69, - 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x19, 0x0a, 0x17, 0x4c, 0x69, - 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x51, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, - 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, - 0x0a, 0x0e, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, - 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x6b, 0x65, 0x79, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x7f, 0x0a, 0x14, 0x53, 0x70, 0x6c, 0x69, + 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x73, 0x70, 0x71, 0x72, 0x22, 0x27, + 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x12, + 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, + 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x8f, 0x01, 0x0a, 0x0c, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x52, 0x05, 0x62, 0x6f, + 0x75, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x72, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6b, 0x72, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, + 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x64, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x39, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x15, 0x0a, 0x06, 0x6e, 0x65, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x6e, 0x65, 0x77, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x1b, 0x0a, - 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x70, - 0x6c, 0x69, 0x74, 0x5f, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, - 0x73, 0x70, 0x6c, 0x69, 0x74, 0x4c, 0x65, 0x66, 0x74, 0x22, 0x52, 0x0a, 0x14, 0x4d, 0x65, 0x72, - 0x67, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x17, 0x0a, 0x07, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x62, 0x61, 0x73, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, - 0x70, 0x65, 0x6e, 0x64, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x67, 0x65, 0x49, 0x64, 0x22, 0x43, 0x0a, - 0x13, 0x4d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x6f, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x6f, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x49, 0x64, 0x22, 0x25, 0x0a, 0x13, 0x44, 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x19, 0x0a, 0x17, 0x44, 0x72, 0x6f, - 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x22, 0x4b, 0x0a, 0x18, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, - 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x2f, 0x0a, 0x09, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x22, 0x25, 0x0a, 0x13, 0x4c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x27, 0x0a, 0x15, 0x55, 0x6e, 0x6c, 0x6f, - 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x02, 0x69, - 0x64, 0x22, 0x4b, 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, - 0x6c, 0x79, 0x12, 0x3a, 0x0a, 0x0f, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x73, - 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x73, 0x70, - 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x0d, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x30, - 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x21, 0x0a, - 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x22, 0x2e, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, - 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, - 0x22, 0x36, 0x0a, 0x14, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x1e, 0x0a, 0x0b, 0x6b, 0x65, 0x79, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6b, - 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x22, 0x26, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4b, - 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, - 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, - 0x2a, 0x2b, 0x0a, 0x0e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x4f, 0x43, 0x4b, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, - 0x0a, 0x09, 0x41, 0x56, 0x41, 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x32, 0xcd, 0x06, - 0x0a, 0x0f, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x12, 0x18, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x73, 0x70, 0x71, - 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, - 0x00, 0x12, 0x40, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x73, - 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, - 0x79, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x4b, 0x65, - 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x1d, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4c, - 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, + 0x12, 0x22, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x19, 0x0a, 0x17, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0x51, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0e, 0x6b, 0x65, 0x79, 0x5f, + 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x22, 0x7f, 0x0a, 0x14, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6e, 0x65, + 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x65, 0x77, 0x49, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x5f, 0x6c, 0x65, + 0x66, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x4c, + 0x65, 0x66, 0x74, 0x22, 0x52, 0x0a, 0x14, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x62, + 0x61, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x61, + 0x73, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x67, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x65, + 0x6e, 0x64, 0x61, 0x67, 0x65, 0x49, 0x64, 0x22, 0x43, 0x0a, 0x13, 0x4d, 0x6f, 0x76, 0x65, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1c, + 0x0a, 0x09, 0x74, 0x6f, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x74, 0x6f, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x64, 0x22, 0x25, 0x0a, 0x13, + 0x44, 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x02, 0x69, 0x64, 0x22, 0x19, 0x0a, 0x17, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, + 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x4b, + 0x0a, 0x18, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x6b, 0x65, + 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x25, 0x0a, 0x13, 0x4c, + 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x02, + 0x69, 0x64, 0x22, 0x27, 0x0a, 0x15, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x4b, 0x0a, 0x0d, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x3a, 0x0a, 0x0f, + 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x6b, 0x65, 0x79, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x30, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, + 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x2e, 0x0a, 0x16, 0x52, 0x65, + 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x22, 0x36, 0x0a, 0x14, 0x52, 0x65, + 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, + 0x6c, 0x79, 0x12, 0x1e, 0x0a, 0x0b, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, + 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x44, 0x22, 0x26, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x2a, 0x2b, 0x0a, 0x0e, 0x4b, 0x65, + 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0a, 0x0a, 0x06, + 0x4c, 0x4f, 0x43, 0x4b, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x56, 0x41, 0x49, + 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x32, 0xcd, 0x06, 0x0a, 0x0f, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x47, + 0x65, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x18, 0x2e, 0x73, 0x70, 0x71, + 0x72, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x40, 0x0a, 0x0c, 0x4c, + 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, + 0x71, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, - 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x3e, 0x0a, - 0x0c, 0x4c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x2e, - 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, - 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x42, 0x0a, - 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, - 0x1b, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, - 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, - 0x00, 0x12, 0x3e, 0x0a, 0x0c, 0x44, 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, - 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, - 0x00, 0x12, 0x53, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x1d, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, - 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, 0x70, - 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x0e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, - 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1b, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, - 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, - 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x40, 0x0a, 0x0d, 0x53, 0x70, - 0x6c, 0x69, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1a, 0x2e, 0x73, 0x70, - 0x71, 0x72, 0x2e, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, - 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x40, 0x0a, 0x0d, - 0x4d, 0x65, 0x72, 0x67, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1a, 0x2e, - 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, - 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x3e, - 0x0a, 0x0c, 0x4d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, - 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, - 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x4d, - 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x12, 0x1c, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, + 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x48, 0x0a, + 0x10, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x73, 0x12, 0x1d, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, + 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x13, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x0c, 0x4c, 0x6f, 0x63, 0x6b, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4c, + 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, + 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1b, 0x2e, 0x73, 0x70, 0x71, 0x72, + 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x0c, 0x44, + 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, + 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x10, 0x44, + 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, + 0x1d, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, + 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, + 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x6c, 0x4b, 0x65, 0x79, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x42, 0x0a, 0x0e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x12, 0x1b, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1a, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, - 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x42, 0x0c, 0x5a, - 0x0a, 0x73, 0x70, 0x71, 0x72, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x33, + 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, 0x65, 0x70, + 0x6c, 0x79, 0x22, 0x00, 0x12, 0x40, 0x0a, 0x0d, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x4b, 0x65, 0x79, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1a, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x53, 0x70, 0x6c, + 0x69, 0x74, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x52, + 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x40, 0x0a, 0x0d, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x4b, + 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1a, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, + 0x65, 0x72, 0x67, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x0c, 0x4d, 0x6f, 0x76, 0x65, + 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, + 0x4d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x73, 0x70, 0x71, 0x72, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x79, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x12, 0x4d, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x6f, + 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x1c, 0x2e, 0x73, 0x70, + 0x71, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x73, 0x70, 0x71, 0x72, + 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x52, 0x65, 0x70, 0x6c, 0x79, 0x22, 0x00, 0x42, 0x0c, 0x5a, 0x0a, 0x73, 0x70, 0x71, 0x72, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1111,7 +1110,7 @@ var file_protos_key_range_proto_enumTypes = make([]protoimpl.EnumInfo, 1) var file_protos_key_range_proto_msgTypes = make([]protoimpl.MessageInfo, 18) var file_protos_key_range_proto_goTypes = []interface{}{ (KeyRangeStatus)(0), // 0: spqr.KeyRangeStatus - (*KeyRange)(nil), // 1: spqr.KeyRange + (*KeyRangeBound)(nil), // 1: spqr.KeyRangeBound (*KeyRangeInfo)(nil), // 2: spqr.KeyRangeInfo (*ListKeyRangeRequest)(nil), // 3: spqr.ListKeyRangeRequest (*ListAllKeyRangesRequest)(nil), // 4: spqr.ListAllKeyRangesRequest @@ -1131,7 +1130,7 @@ var file_protos_key_range_proto_goTypes = []interface{}{ (*GetKeyRangeRequest)(nil), // 18: spqr.GetKeyRangeRequest } var file_protos_key_range_proto_depIdxs = []int32{ - 1, // 0: spqr.KeyRangeInfo.key_range:type_name -> spqr.KeyRange + 1, // 0: spqr.KeyRangeInfo.bound:type_name -> spqr.KeyRangeBound 2, // 1: spqr.CreateKeyRangeRequest.key_range_info:type_name -> spqr.KeyRangeInfo 2, // 2: spqr.DropAllKeyRangesResponse.key_range:type_name -> spqr.KeyRangeInfo 2, // 3: spqr.KeyRangeReply.key_ranges_info:type_name -> spqr.KeyRangeInfo @@ -1173,7 +1172,7 @@ func file_protos_key_range_proto_init() { } if !protoimpl.UnsafeEnabled { file_protos_key_range_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KeyRange); i { + switch v := v.(*KeyRangeBound); i { case 0: return &v.state case 1: diff --git a/protos/key_range.proto b/protos/key_range.proto index 93231786e..6b2ce666d 100644 --- a/protos/key_range.proto +++ b/protos/key_range.proto @@ -24,13 +24,14 @@ enum KeyRangeStatus { AVAILABLE = 1; } -message KeyRange { - string lower_bound = 1; +message KeyRangeBound { + repeated bytes values = 1; } // key range info is mapped to shard message KeyRangeInfo { - KeyRange key_range = 1; + KeyRangeBound bound = 1; + string krid = 2; string shardId = 3; string distributionId = 4; diff --git a/qdb/etcdqdb.go b/qdb/etcdqdb.go index bfe4cc044..f03f128a8 100644 --- a/qdb/etcdqdb.go +++ b/qdb/etcdqdb.go @@ -107,7 +107,7 @@ func transferTxNodePath(key string) string { // TODO : unit tests func (q *EtcdQDB) CreateKeyRange(ctx context.Context, keyRange *KeyRange) error { spqrlog.Zero.Debug(). - Bytes("lower-bound", keyRange.LowerBound). + Bytes("lower-bound", keyRange.LowerBound[0]). Str("shard-id", keyRange.ShardID). Str("distribution-id", keyRange.DistributionId). Str("key-range-id", keyRange.KeyRangeID). @@ -169,7 +169,7 @@ func (q *EtcdQDB) GetKeyRange(ctx context.Context, id string) (*KeyRange, error) // TODO : unit tests func (q *EtcdQDB) UpdateKeyRange(ctx context.Context, keyRange *KeyRange) error { spqrlog.Zero.Debug(). - Bytes("lower-bound", keyRange.LowerBound). + Bytes("lower-bound", keyRange.LowerBound[0]). Str("shard-id", keyRange.ShardID). Str("distribution-id", keyRange.KeyRangeID). Str("key-range-id", keyRange.KeyRangeID). diff --git a/qdb/memqdb_test.go b/qdb/memqdb_test.go index e8edf24ad..739bdfd6c 100644 --- a/qdb/memqdb_test.go +++ b/qdb/memqdb_test.go @@ -19,7 +19,7 @@ var mockShard = &qdb.Shard{ Hosts: []string{"host1", "host2"}, } var mockKeyRange = &qdb.KeyRange{ - LowerBound: []byte{1, 2}, + LowerBound: [][]byte{[]byte{1, 2}}, ShardID: mockShard.ID, KeyRangeID: "key_range_id", } @@ -191,14 +191,14 @@ func TestKeyRanges(t *testing.T) { assert.NoError(err) assert.NoError(memqdb.CreateKeyRange(ctx, &qdb.KeyRange{ - LowerBound: []byte("1111"), + LowerBound: [][]byte{[]byte("1111")}, ShardID: "sh1", KeyRangeID: "krid1", DistributionId: "ds1", })) assert.Error(memqdb.CreateKeyRange(ctx, &qdb.KeyRange{ - LowerBound: []byte("1111"), + LowerBound: [][]byte{[]byte("1111")}, ShardID: "sh1", KeyRangeID: "krid2", DistributionId: "dserr", @@ -222,7 +222,7 @@ func Test_MemQDB_GetKeyRange(t *testing.T) { assert.NoError(err) keyRange1 := qdb.KeyRange{ - LowerBound: []byte("1111"), + LowerBound: [][]byte{[]byte("1111")}, ShardID: "sh1", KeyRangeID: "krid1", DistributionId: "ds1", @@ -230,7 +230,7 @@ func Test_MemQDB_GetKeyRange(t *testing.T) { assert.NoError(memqdb.CreateKeyRange(ctx, &keyRange1)) keyRange2 := qdb.KeyRange{ - LowerBound: []byte("1111"), + LowerBound: [][]byte{[]byte("1111")}, ShardID: "sh1", KeyRangeID: "krid2", DistributionId: "ds2", diff --git a/qdb/models.go b/qdb/models.go index 57c707be5..934ce067a 100644 --- a/qdb/models.go +++ b/qdb/models.go @@ -6,10 +6,10 @@ type ShardKey struct { } type KeyRange struct { - LowerBound []byte `json:"from"` - ShardID string `json:"shard_id"` - KeyRangeID string `json:"key_range_id"` - DistributionId string `json:"distribution_id"` + LowerBound [][]byte `json:"from"` + ShardID string `json:"shard_id"` + KeyRangeID string `json:"key_range_id"` + DistributionId string `json:"distribution_id"` } type MoveKeyRangeStatus string @@ -69,9 +69,11 @@ func NewShard(ID string, hosts []string) *Shard { } var ( - ColumnTypeVarchar = "varchar" - ColumnTypeInteger = "integer" - ColumnTypeUinteger = "uinteger" + ColumnTypeVarchar = "varchar" + ColumnTypeVarcharHashed = "varchar hashed" + ColumnTypeVarcharDeprecated = "_varchar" + ColumnTypeInteger = "integer" + ColumnTypeUinteger = "uinteger" ) type DistributionKeyEntry struct { diff --git a/qdb/ops/ops.go b/qdb/ops/ops.go index ded72ae3d..3bbc2f658 100644 --- a/qdb/ops/ops.go +++ b/qdb/ops/ops.go @@ -5,6 +5,7 @@ import ( "github.com/pg-sharding/spqr/pkg/models/kr" "github.com/pg-sharding/spqr/pkg/models/spqrerror" + "github.com/pg-sharding/spqr/pkg/spqrlog" "github.com/pg-sharding/spqr/qdb" ) @@ -29,7 +30,10 @@ func CreateKeyRangeWithChecks(ctx context.Context, qdb qdb.QDB, keyRange *kr.Key } for _, v := range existsKrids { - if kr.CmpRangesEqual(keyRange.LowerBound, v.LowerBound) { + + spqrlog.Zero.Info().Bytes("types", v.LowerBound[0]).Msg("adding key range") + eph := kr.KeyRangeFromBytes(v.LowerBound, keyRange.ColumnTypes) + if kr.CmpRangesEqual(keyRange.LowerBound, eph.LowerBound, keyRange.ColumnTypes) { return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "key range %v intersects with key range %v in QDB", keyRange.ID, v.KeyRangeID) } } @@ -58,7 +62,9 @@ func ModifyKeyRangeWithChecks(ctx context.Context, qdb qdb.QDB, keyRange *kr.Key // update req continue } - if kr.CmpRangesEqual(keyRange.LowerBound, v.LowerBound) { + + eph := kr.KeyRangeFromBytes(v.LowerBound, keyRange.ColumnTypes) + if kr.CmpRangesEqual(keyRange.LowerBound, eph.LowerBound, keyRange.ColumnTypes) { return spqrerror.Newf(spqrerror.SPQR_KEYRANGE_ERROR, "key range %v intersects with key range %v in QDB", keyRange.ID, v.KeyRangeID) } } diff --git a/router/client/client.go b/router/client/client.go index 395552446..ca812f2e0 100644 --- a/router/client/client.go +++ b/router/client/client.go @@ -954,7 +954,7 @@ func (cl *PsqlClient) ReplyErr(e error) error { case *spqrerror.SpqrError: return cl.ReplyErrMsg(er.Error(), er.ErrorCode) default: - return cl.ReplyErrMsg(e.Error(), "SPQRU") + return cl.ReplyErrMsg(e.Error(), spqrerror.SPQR_UNEXPECTED) } } diff --git a/router/grpc/qrouter.go b/router/grpc/qrouter.go index 3849ebb96..f5f2194ca 100644 --- a/router/grpc/qrouter.go +++ b/router/grpc/qrouter.go @@ -231,7 +231,11 @@ func (l *LocalQrouterServer) DropShardingRules(ctx context.Context, request *pro // TODO : unit tests func (l *LocalQrouterServer) CreateKeyRange(ctx context.Context, request *protos.CreateKeyRangeRequest) (*protos.ModifyReply, error) { - err := l.mgr.CreateKeyRange(ctx, kr.KeyRangeFromProto(request.KeyRangeInfo)) + ds, err := l.mgr.GetDistribution(ctx, request.KeyRangeInfo.DistributionId) + if err != nil { + return nil, err + } + err = l.mgr.CreateKeyRange(ctx, kr.KeyRangeFromProto(request.KeyRangeInfo, ds.ColTypes)) if err != nil { return nil, err } @@ -314,7 +318,7 @@ func (l *LocalQrouterServer) SplitKeyRange(ctx context.Context, request *protos. if err := l.mgr.Split(ctx, &kr.SplitKeyRange{ Krid: request.NewId, SourceID: request.SourceId, - Bound: request.Bound, + Bound: [][]byte{request.Bound}, // TODO: fix SplitLeft: request.SplitLeft, }); err != nil { return nil, err diff --git a/router/instance/etcd.go b/router/instance/etcd.go index 9cac3f4e7..7b9144ac0 100644 --- a/router/instance/etcd.go +++ b/router/instance/etcd.go @@ -41,7 +41,7 @@ func (e *EtcdMetadataBootstraper) InitializeMetadata(ctx context.Context, r Rout } for _, ckr := range krs { - if err := r.Console().Mgr().CreateKeyRange(ctx, kr.KeyRangeFromDB(ckr)); err != nil { + if err := r.Console().Mgr().CreateKeyRange(ctx, kr.KeyRangeFromDB(ckr, d.ColTypes)); err != nil { spqrlog.Zero.Error().Err(err).Msg("failed to initialize instance") return err } diff --git a/router/mock/qrouter/mock_qrouter.go b/router/mock/qrouter/mock_qrouter.go index ad1218e89..fa28de1c6 100644 --- a/router/mock/qrouter/mock_qrouter.go +++ b/router/mock/qrouter/mock_qrouter.go @@ -54,7 +54,7 @@ func (mr *MockQueryRouterMockRecorder) DataShardsRoutes() *gomock.Call { } // DeparseKeyWithRangesInternal mocks base method. -func (m *MockQueryRouter) DeparseKeyWithRangesInternal(ctx context.Context, key string, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) { +func (m *MockQueryRouter) DeparseKeyWithRangesInternal(ctx context.Context, key []interface{}, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "DeparseKeyWithRangesInternal", ctx, key, krs) ret0, _ := ret[0].(*routingstate.DataShardRoute) diff --git a/router/qrouter/proxy_routing.go b/router/qrouter/proxy_routing.go index bb1200e84..198ac64c8 100644 --- a/router/qrouter/proxy_routing.go +++ b/router/qrouter/proxy_routing.go @@ -1,10 +1,15 @@ package qrouter import ( + "bytes" "context" + "encoding/binary" "fmt" + "strconv" "strings" + "github.com/pg-sharding/spqr/pkg/meta" + "github.com/pg-sharding/spqr/pkg/models/distributions" "github.com/pg-sharding/spqr/pkg/models/spqrerror" "github.com/pg-sharding/spqr/pkg/config" @@ -43,7 +48,7 @@ type RoutingMetadataContext struct { // SELECT * FROM a join b WHERE a.c1 = and a.c2 = // can be routed with different rules rels map[RelationFQN]struct{} - exprs map[RelationFQN]map[string][]string + exprs map[RelationFQN]map[string][]interface{} // cached CTE names cteNames map[string]struct{} @@ -53,6 +58,8 @@ type RoutingMetadataContext struct { // rarg:{range_var:{relname:"t2" inh:true relpersistence:"p" alias:{aliasname:"b"} tableAliases map[string]RelationFQN + distributions map[RelationFQN]*distributions.Distribution + params [][]byte paramsFormatCodes []int16 // TODO: include client ops and metadata here @@ -60,11 +67,12 @@ type RoutingMetadataContext struct { func NewRoutingMetadataContext(params [][]byte, paramsFormatCodes []int16) *RoutingMetadataContext { meta := &RoutingMetadataContext{ - rels: map[RelationFQN]struct{}{}, - cteNames: map[string]struct{}{}, - tableAliases: map[string]RelationFQN{}, - exprs: map[RelationFQN]map[string][]string{}, - params: params, + rels: map[RelationFQN]struct{}{}, + cteNames: map[string]struct{}{}, + tableAliases: map[string]RelationFQN{}, + exprs: map[RelationFQN]map[string][]interface{}{}, + distributions: map[RelationFQN]*distributions.Distribution{}, + params: params, } // https://github.com/postgres/postgres/blob/master/src/backend/tcop/pquery.c#L635-L658 if len(paramsFormatCodes) > 1 { @@ -88,23 +96,36 @@ func NewRoutingMetadataContext(params [][]byte, paramsFormatCodes []int16) *Rout return meta } +func (m *RoutingMetadataContext) GetRelationDistribution(ctx context.Context, mgr meta.EntityMgr, resolvedRelation RelationFQN) (*distributions.Distribution, error) { + if res, ok := m.distributions[resolvedRelation]; ok { + return res, nil + } + ds, err := mgr.GetRelationDistribution(ctx, resolvedRelation.RelationName) + + if err != nil { + return nil, err + } + m.distributions[resolvedRelation] = ds + return ds, nil +} + func (meta *RoutingMetadataContext) RFQNIsCTE(resolvedRelation RelationFQN) bool { _, ok := meta.cteNames[resolvedRelation.RelationName] return len(resolvedRelation.SchemaName) == 0 && ok } // TODO : unit tests -func (meta *RoutingMetadataContext) RecordConstExpr(resolvedRelation RelationFQN, colname string, expr string) error { +func (meta *RoutingMetadataContext) RecordConstExpr(resolvedRelation RelationFQN, colname string, expr interface{}) error { if meta.RFQNIsCTE(resolvedRelation) { // CTE, skip return nil } meta.rels[resolvedRelation] = struct{}{} if _, ok := meta.exprs[resolvedRelation]; !ok { - meta.exprs[resolvedRelation] = map[string][]string{} + meta.exprs[resolvedRelation] = map[string][]interface{}{} } if _, ok := meta.exprs[resolvedRelation][colname]; !ok { - meta.exprs[resolvedRelation][colname] = make([]string, 0) + meta.exprs[resolvedRelation][colname] = make([]interface{}, 0) } meta.exprs[resolvedRelation][colname] = append(meta.exprs[resolvedRelation][colname], expr) return nil @@ -151,21 +172,17 @@ func (qr *ProxyQrouter) DeparseExprShardingEntries(expr lyx.Node, meta *RoutingM } // TODO : unit tests -func (qr *ProxyQrouter) DeparseKeyWithRangesInternal(_ context.Context, key string, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) { - spqrlog.Zero.Debug(). - Str("key", key). - Msg("checking key") - +func (qr *ProxyQrouter) DeparseKeyWithRangesInternal(_ context.Context, key []interface{}, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) { spqrlog.Zero.Debug(). - Str("key", key). + Interface("key", key[0]). Int("key-ranges-count", len(krs)). Msg("checking key with key ranges") var matched_krkey *kr.KeyRange = nil for _, krkey := range krs { - if kr.CmpRangesLessEqual(krkey.LowerBound, []byte(key)) && - (matched_krkey == nil || kr.CmpRangesLessEqual(matched_krkey.LowerBound, krkey.LowerBound)) { + if kr.CmpRangesLessEqual(krkey.LowerBound, key, krkey.ColumnTypes) && + (matched_krkey == nil || kr.CmpRangesLessEqual(matched_krkey.LowerBound, krkey.LowerBound, krkey.ColumnTypes)) { matched_krkey = krkey } } @@ -184,73 +201,189 @@ func (qr *ProxyQrouter) DeparseKeyWithRangesInternal(_ context.Context, key stri return nil, FailedToFindKeyRange } -func (qr *ProxyQrouter) RecordDistributionKeyColumnValueOnRFQN(meta *RoutingMetadataContext, resolvedRelation RelationFQN, colname, value string) error { - +func (qr *ProxyQrouter) GetDistributionKeyOffsetType(meta *RoutingMetadataContext, resolvedRelation RelationFQN, colname string) (int, string) { /* do not process non-distributed relations or columns not from relation distribution key */ - if ds, err := qr.Mgr().GetRelationDistribution(context.TODO(), resolvedRelation.RelationName); err != nil { - return nil - } else { - // TODO: optimize - ok := false - for _, c := range ds.Relations[resolvedRelation.RelationName].DistributionKey { - if c.Column == colname { - ok = true - break - } + + ds, err := meta.GetRelationDistribution(context.TODO(), qr.Mgr(), resolvedRelation) + if err != nil { + return -1, "" + } + // TODO: optimize + for ind, c := range ds.Relations[resolvedRelation.RelationName].DistributionKey { + if c.Column == colname { + return ind, ds.ColTypes[ind] } - if !ok { - // some junk column - return nil + } + return -1, "" +} + +func (qr *ProxyQrouter) RecordDistributionKeyColumnValueOnRFQN(meta *RoutingMetadataContext, resolvedRelation RelationFQN, colname string, value interface{}) error { + /* do not process non-distributed relations or columns not from relation distribution key */ + + ds, err := meta.GetRelationDistribution(context.TODO(), qr.Mgr(), resolvedRelation) + if err != nil { + return err + } + // TODO: optimize + ok := false + for _, c := range ds.Relations[resolvedRelation.RelationName].DistributionKey { + if c.Column == colname { + ok = true + break } } + if !ok { + // some junk column + return nil + } + // will not work not ints return meta.RecordConstExpr(resolvedRelation, colname, value) } -// TODO : unit tests -func (qr *ProxyQrouter) RecordDistributionKeyExprOnRFQN(meta *RoutingMetadataContext, resolvedRelation RelationFQN, colname string, expr lyx.Node) error { - switch e := expr.(type) { +func (qr *ProxyQrouter) processConstExpr(alias, colname string, expr lyx.Node, meta *RoutingMetadataContext) error { + resolvedRelation, err := meta.ResolveRelationByAlias(alias) + if err != nil { + // failed to resolve relation, skip column + return nil + } + + return qr.processConstExprOnRFQN(resolvedRelation, colname, expr, meta) +} + +func (qr *ProxyQrouter) processConstExprOnRFQN(resolvedRelation RelationFQN, colname string, expr lyx.Node, meta *RoutingMetadataContext) error { + off, tp := qr.GetDistributionKeyOffsetType(meta, resolvedRelation, colname) + if off == -1 { + // column not from distr key + return nil + } + + /* simple key-value pair */ + switch rght := expr.(type) { case *lyx.ParamRef: - if e.Number > len(meta.params) { - return ComplexQuery - } + if rght.Number <= len(meta.params) { + // TODO: switch column type here + fc := meta.paramsFormatCodes[rght.Number-1] + + switch fc { + case xproto.FormatCodeBinary: + switch tp { + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeVarchar: + return meta.RecordConstExpr(resolvedRelation, colname, string(meta.params[rght.Number-1])) + case qdb.ColumnTypeInteger: + + var num int64 + var err error + + buf := bytes.NewBuffer(meta.params[rght.Number-1]) + + if len(meta.params[rght.Number-1]) == 4 { + var tmpnum int32 + err = binary.Read(buf, binary.BigEndian, &tmpnum) + num = int64(tmpnum) + } else { + err = binary.Read(buf, binary.BigEndian, &num) + } + if err != nil { + return err + } - // switch parameter format code and convert into proper representation + return meta.RecordConstExpr(resolvedRelation, colname, num) + case qdb.ColumnTypeUinteger: - var routeParam []byte - fc := meta.paramsFormatCodes[e.Number-1] + var num uint64 + var err error + + buf := bytes.NewBuffer(meta.params[rght.Number-1]) + + if len(meta.params[rght.Number-1]) == 4 { + var tmpnum uint32 + err = binary.Read(buf, binary.BigEndian, &tmpnum) + num = uint64(tmpnum) + } else { + err = binary.Read(buf, binary.BigEndian, &num) + } + if err != nil { + return err + } + + return meta.RecordConstExpr(resolvedRelation, colname, num) + } + case xproto.FormatCodeText: + switch tp { + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeVarchar: + return meta.RecordConstExpr(resolvedRelation, colname, string(meta.params[rght.Number-1])) + case qdb.ColumnTypeInteger: + num, err := strconv.ParseInt(string(meta.params[rght.Number-1]), 10, 64) + if err != nil { + return err + } + return meta.RecordConstExpr(resolvedRelation, colname, num) + case qdb.ColumnTypeUinteger: + num, err := strconv.ParseUint(string(meta.params[rght.Number-1]), 10, 64) + if err != nil { + return err + } + return meta.RecordConstExpr(resolvedRelation, colname, num) + } + default: + // ??? protoc violation + } - switch fc { - case xproto.FormatCodeBinary: - // TODO: here we need to invoke out function for convertion - // actually, we need to convert everything to binary format - case xproto.FormatCodeText: - routeParam = meta.params[e.Number-1] - default: - // ??? protoc violation } - return qr.RecordDistributionKeyColumnValueOnRFQN(meta, resolvedRelation, colname, string(routeParam)) + return fmt.Errorf("expression is out of range") + // else error out? case *lyx.AExprSConst: - return qr.RecordDistributionKeyColumnValueOnRFQN(meta, resolvedRelation, colname, string(e.Value)) + switch tp { + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeVarchar: + return meta.RecordConstExpr(resolvedRelation, colname, rght.Value) + case qdb.ColumnTypeInteger: + num, err := strconv.ParseInt(rght.Value, 10, 64) + if err != nil { + return err + } + return meta.RecordConstExpr(resolvedRelation, colname, num) + case qdb.ColumnTypeUinteger: + num, err := strconv.ParseUint(rght.Value, 10, 64) + if err != nil { + return err + } + return meta.RecordConstExpr(resolvedRelation, colname, num) + default: + return fmt.Errorf("expression with incorrect type") + } case *lyx.AExprIConst: - val := fmt.Sprintf("%d", e.Value) - return qr.RecordDistributionKeyColumnValueOnRFQN(meta, resolvedRelation, colname, string(val)) + switch tp { + case qdb.ColumnTypeVarcharDeprecated: + fallthrough + case qdb.ColumnTypeVarcharHashed: + fallthrough + case qdb.ColumnTypeVarchar: + return fmt.Errorf("expression with incorrect type") + case qdb.ColumnTypeInteger: + return meta.RecordConstExpr(resolvedRelation, colname, int64(rght.Value)) + case qdb.ColumnTypeUinteger: + return meta.RecordConstExpr(resolvedRelation, colname, uint64(rght.Value)) + default: + return fmt.Errorf("expression with incorrect type") + } default: - return ComplexQuery - } -} - -func (qr *ProxyQrouter) RecordDistributionKeyColumnValue(meta *RoutingMetadataContext, alias, colname, value string) error { - - resolvedRelation, err := meta.ResolveRelationByAlias(alias) - if err != nil { - // failed to resolve relation, skip column - return nil + return fmt.Errorf("expression is not const") } - - return qr.RecordDistributionKeyColumnValueOnRFQN(meta, resolvedRelation, colname, value) } // routeByClause de-parses sharding column-value pair from Where clause of the query @@ -274,39 +407,20 @@ func (qr *ProxyQrouter) routeByClause(ctx context.Context, expr lyx.Node, meta * /* simple key-value pair */ switch rght := texpr.Right.(type) { - case *lyx.ParamRef: - if rght.Number <= len(meta.params) { - if err := qr.RecordDistributionKeyColumnValue(meta, alias, colname, string(meta.params[rght.Number-1])); err != nil { - return err - } - } + case *lyx.ParamRef, *lyx.AExprSConst, *lyx.AExprIConst: // else error out? - case *lyx.AExprSConst: - // TBD: postpone routing from here to root of parsing tree - if err := qr.RecordDistributionKeyColumnValue(meta, alias, colname, rght.Value); err != nil { - return err - } - case *lyx.AExprIConst: + // TBD: postpone routing from here to root of parsing tree // maybe expimely inefficient. Will be fixed in SPQR-2.0 - if err := qr.RecordDistributionKeyColumnValue(meta, alias, colname, fmt.Sprintf("%d", rght.Value)); err != nil { + if err := qr.processConstExpr(alias, colname, rght, meta); err != nil { return err } + case *lyx.AExprList: if len(rght.List) != 0 { expr := rght.List[0] - switch bexpr := expr.(type) { - case *lyx.AExprSConst: - // TBD: postpone routing from here to root of parsing tree - if err := qr.RecordDistributionKeyColumnValue(meta, alias, colname, bexpr.Value); err != nil { - return err - } - case *lyx.AExprIConst: - // TBD: postpone routing from here to root of parsing tree - // maybe expimely inefficient. Will be fixed in SPQR-2.0 - if err := qr.RecordDistributionKeyColumnValue(meta, alias, colname, fmt.Sprintf("%d", bexpr.Value)); err != nil { - return err - } + if err := qr.processConstExpr(alias, colname, expr, meta); err != nil { + return err } } case *lyx.FuncApplication: @@ -590,7 +704,7 @@ func (qr *ProxyQrouter) deparseShardingMapping( if tlUsable { for i := range offsets { - _ = qr.RecordDistributionKeyExprOnRFQN(meta, rfqn, insertCols[offsets[i]], targetList[offsets[i]]) + _ = qr.processConstExprOnRFQN(rfqn, insertCols[offsets[i]], targetList[offsets[i]], meta) } } @@ -615,10 +729,9 @@ func (qr *ProxyQrouter) deparseShardingMapping( if vlUsable { for i := range offsets { - _ = qr.RecordDistributionKeyExprOnRFQN(meta, rfqn, insertCols[offsets[i]], valList[offsets[i]]) + _ = qr.processConstExprOnRFQN(rfqn, insertCols[offsets[i]], valList[offsets[i]], meta) } } - } } @@ -909,7 +1022,7 @@ func (qr *ProxyQrouter) routeWithRules(ctx context.Context, stmt lyx.Node, sph s ok := true - var hashedKeys [][]byte + compositeKey := make([]interface{}, len(distrKey)) // TODO: multi-column routing. This works only for one-dim routing for i := 0; i < len(distrKey); i++ { @@ -928,16 +1041,17 @@ func (qr *ProxyQrouter) routeWithRules(ctx context.Context, stmt lyx.Node, sph s break } - hashedKeys = make([][]byte, len(vals)) for i, val := range vals { - hashedKeys[i], err = hashfunction.ApplyHashFunction([]byte(val), hf) - spqrlog.Zero.Debug().Str("key", meta.exprs[rfqn][col][i]).Str("hashed key", string(hashedKeys[i])).Msg("applying hash function on key") + compositeKey[i], err = hashfunction.ApplyHashFunction(val, ds.ColTypes[i], hf) + spqrlog.Zero.Debug().Interface("key", meta.exprs[rfqn][col][i]).Interface("hashed key", compositeKey[i]).Msg("applying hash function on key") if err != nil { spqrlog.Zero.Debug().Err(err).Msg("failed to apply hash function") ok = false break } + // only works for one value + break } } @@ -945,25 +1059,26 @@ func (qr *ProxyQrouter) routeWithRules(ctx context.Context, stmt lyx.Node, sph s // skip this relation continue } - for _, hashedKey := range hashedKeys { - currroute, err := qr.DeparseKeyWithRangesInternal(ctx, string(hashedKey), krs) - if err != nil { - route_err = err - spqrlog.Zero.Debug().Err(route_err).Msg("temporarily skip the route error") - continue - } - - spqrlog.Zero.Debug(). - Interface("currroute", currroute). - Str("table", rfqn.RelationName). - Msg("calculated route for table/cols") - route = routingstate.Combine(route, routingstate.ShardMatchState{ - Route: currroute, - TargetSessionAttrs: tsa, - }) + currroute, err := qr.DeparseKeyWithRangesInternal(ctx, compositeKey, krs) + if err != nil { + route_err = err + spqrlog.Zero.Debug().Err(route_err).Msg("temporarily skip the route error") + continue } + + spqrlog.Zero.Debug(). + Interface("currroute", currroute). + Str("table", rfqn.RelationName). + Msg("calculated route for table/cols") + + route = routingstate.Combine(route, routingstate.ShardMatchState{ + Route: currroute, + TargetSessionAttrs: tsa, + }) + } + if route == nil && route_err != nil { return nil, route_err } diff --git a/router/qrouter/proxy_routing_test.go b/router/qrouter/proxy_routing_test.go index e07b5ceb3..53c3a1f2f 100644 --- a/router/qrouter/proxy_routing_test.go +++ b/router/qrouter/proxy_routing_test.go @@ -131,6 +131,9 @@ func TestComment(t *testing.T) { _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ ID: distribution, + ColTypes: []string{ + qdb.ColumnTypeInteger, + }, Relations: map[string]*qdb.DistributedRelation{ "xx": { Name: "xx", @@ -143,21 +146,31 @@ func TestComment(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - DistributionId: distribution, - KeyRangeID: "id1", - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + LowerBound: kr.KeyRangeBound{ + int64(1), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - KeyRangeID: "id2", - DistributionId: distribution, - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: kr.KeyRangeBound{ + int64(11), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) @@ -188,7 +201,10 @@ func TestComment(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -222,6 +238,8 @@ func TestCTE(t *testing.T) { _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ ID: distribution, + + ColTypes: []string{qdb.ColumnTypeInteger}, Relations: map[string]*qdb.DistributedRelation{ "t": { Name: "t", @@ -234,21 +252,29 @@ func TestCTE(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - DistributionId: distribution, - KeyRangeID: "id1", - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", LowerBound: kr.KeyRangeBound{ + int64(1), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - DistributionId: distribution, - KeyRangeID: "id2", - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", LowerBound: kr.KeyRangeBound{ + int64(11), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) @@ -289,7 +315,11 @@ func TestCTE(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -313,7 +343,10 @@ func TestCTE(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -341,27 +374,30 @@ func TestCTE(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", }, }, - { - query: ` - WITH xxxx AS ( - SELECT * from t where i = 1 - ), - zzzz AS ( - UPDATE t - SET a = 0 - WHERE i = 12 - ) - SELECT * FROM xxxx; - `, - err: nil, - exp: routingstate.SkipRoutingState{}, - }, + // { + // query: ` + // WITH xxxx AS ( + // SELECT * from t where i = 1 + // ), + // zzzz AS ( + // UPDATE t + // SET a = 0 + // WHERE i = 12 + // ) + // SELECT * FROM xxxx; + // `, + // err: nil, + // exp: routingstate.SkipRoutingState{}, + // }, { query: ` WITH xxxx AS ( @@ -384,7 +420,9 @@ func TestCTE(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{int64(1)}, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -421,6 +459,9 @@ func TestSingleShard(t *testing.T) { _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ ID: distribution, + ColTypes: []string{ + qdb.ColumnTypeInteger, + }, Relations: map[string]*qdb.DistributedRelation{ "t": { Name: "t", @@ -465,21 +506,31 @@ func TestSingleShard(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - DistributionId: distribution, - KeyRangeID: "id1", - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + LowerBound: kr.KeyRangeBound{ + int64(1), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - DistributionId: distribution, - KeyRangeID: "id2", - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: kr.KeyRangeBound{ + int64(11), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB()) assert.NoError(err) @@ -499,6 +550,55 @@ func TestSingleShard(t *testing.T) { assert.NoError(err) for _, tt := range []tcase{ + { + query: "SELECT * FROM xxtt1 a WHERE a.i = 21 and w_idj + w_idi != 0;", + exp: routingstate.ShardMatchState{ + Route: &routingstate.DataShardRoute{ + Shkey: kr.ShardKey{ + Name: "sh2", + }, + Matchedkr: &kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }, + }, + TargetSessionAttrs: "any", + }, + err: nil, + }, + + { + query: "SELECT * FROM xxtt1 a WHERE a.i = '21' and w_idj + w_idi != 0;", + exp: routingstate.ShardMatchState{ + Route: &routingstate.DataShardRoute{ + Shkey: kr.ShardKey{ + Name: "sh2", + }, + Matchedkr: &kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }, + }, + TargetSessionAttrs: "any", + }, + err: nil, + }, { query: ` @@ -519,7 +619,14 @@ func TestSingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + + int64(1), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -545,7 +652,14 @@ func TestSingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + + int64(1), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -562,7 +676,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -581,26 +701,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), - }, - }, - TargetSessionAttrs: "any", - }, - err: nil, - }, + LowerBound: []interface{}{ + int64(11), + }, - { - query: "SELECT * FROM xxtt1 a WHERE a.i = 21 and w_idj + w_idi != 0;", - exp: routingstate.ShardMatchState{ - Route: &routingstate.DataShardRoute{ - Shkey: kr.ShardKey{ - Name: "sh2", - }, - Matchedkr: &kr.KeyRange{ - ShardID: "sh2", - Distribution: distribution, - ID: "id2", - LowerBound: []byte("11"), + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -618,7 +725,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -637,7 +750,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -659,7 +778,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -678,7 +803,13 @@ func TestSingleShard(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -697,7 +828,11 @@ func TestSingleShard(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{int64(11)}, + + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, }, }, TargetSessionAttrs: "any", @@ -715,7 +850,11 @@ func TestSingleShard(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + + LowerBound: []interface{}{ + int64(11), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -748,7 +887,8 @@ func TestInsertOffsets(t *testing.T) { distribution := "dd" _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ - ID: distribution, + ID: distribution, + ColTypes: []string{qdb.ColumnTypeInteger}, Relations: map[string]*qdb.DistributedRelation{ "xx": { Name: "xx", @@ -777,21 +917,25 @@ func TestInsertOffsets(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - KeyRangeID: "id1", - DistributionId: distribution, - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + LowerBound: []interface{}{int64(1)}, + + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - DistributionId: distribution, - KeyRangeID: "id2", - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + LowerBound: []interface{}{int64(11)}, + + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) @@ -823,7 +967,11 @@ func TestInsertOffsets(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + LowerBound: []interface{}{ + int64(11), + }, }, }, TargetSessionAttrs: "any", @@ -833,7 +981,7 @@ func TestInsertOffsets(t *testing.T) { { query: ` - INSERT INTO xxtt1 (j, i, w_id) VALUES(2121221, -211212, '21'); + INSERT INTO xxtt1 (j, i, w_id) VALUES(2121221, -211212, 21); `, exp: routingstate.ShardMatchState{ Route: &routingstate.DataShardRoute{ @@ -844,7 +992,11 @@ func TestInsertOffsets(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + LowerBound: []interface{}{ + int64(11), + }, }, }, TargetSessionAttrs: "any", @@ -854,7 +1006,7 @@ func TestInsertOffsets(t *testing.T) { { query: ` - INSERT INTO "people" ("first_name","last_name","email","id") VALUES ('John','Smith','','1') RETURNING "id"`, + INSERT INTO "people" ("first_name","last_name","email","id") VALUES ('John','Smith','',1) RETURNING "id"`, exp: routingstate.ShardMatchState{ Route: &routingstate.DataShardRoute{ Shkey: kr.ShardKey{ @@ -864,7 +1016,11 @@ func TestInsertOffsets(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + LowerBound: []interface{}{ + int64(1), + }, }, }, TargetSessionAttrs: "any", @@ -884,7 +1040,11 @@ func TestInsertOffsets(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + LowerBound: []interface{}{ + int64(11), + }, }, }, TargetSessionAttrs: "any", @@ -903,7 +1063,11 @@ func TestInsertOffsets(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + LowerBound: []interface{}{ + int64(1), + }, }, }, TargetSessionAttrs: "any", @@ -937,7 +1101,7 @@ func TestJoins(t *testing.T) { _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ ID: distribution, - ColTypes: []string{qdb.ColumnTypeVarchar}, + ColTypes: []string{qdb.ColumnTypeInteger}, Relations: map[string]*qdb.DistributedRelation{ "sshjt1": { Name: "sshjt1", @@ -966,21 +1130,23 @@ func TestJoins(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - KeyRangeID: "id1", - DistributionId: distribution, - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + LowerBound: []interface{}{int64(11)}, + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - KeyRangeID: "id2", - DistributionId: distribution, - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: []interface{}{int64(11)}, + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) @@ -1009,7 +1175,10 @@ func TestJoins(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -1028,7 +1197,10 @@ func TestJoins(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -1078,7 +1250,8 @@ func TestUnnest(t *testing.T) { distribution := "dd" _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ - ID: distribution, + ID: distribution, + ColTypes: []string{qdb.ColumnTypeInteger}, Relations: map[string]*qdb.DistributedRelation{ "xxtt1": { Name: "xxtt1", @@ -1091,21 +1264,23 @@ func TestUnnest(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - KeyRangeID: "id1", - DistributionId: distribution, - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + LowerBound: []interface{}{int64(11)}, + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - DistributionId: distribution, - KeyRangeID: "id2", - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: []interface{}{int64(11)}, + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) @@ -1137,7 +1312,11 @@ func TestUnnest(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -1156,7 +1335,11 @@ func TestUnnest(t *testing.T) { ShardID: "sh2", ID: "id2", Distribution: distribution, - LowerBound: []byte("11"), + LowerBound: []interface{}{ + int64(11), + }, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -1189,7 +1372,8 @@ func TestCopySingleShard(t *testing.T) { distribution := "dd" _ = db.CreateDistribution(context.TODO(), &qdb.Distribution{ - ID: distribution, + ID: distribution, + ColTypes: []string{qdb.ColumnTypeInteger}, Relations: map[string]*qdb.DistributedRelation{ "xx": { Name: "xx", @@ -1202,21 +1386,25 @@ func TestCopySingleShard(t *testing.T) { }, }) - err := db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh1", - DistributionId: distribution, - KeyRangeID: "id1", - LowerBound: []byte("1"), - }) + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution, + ID: "id1", + LowerBound: []interface{}{int64(1)}, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) - err = db.CreateKeyRange(context.TODO(), &qdb.KeyRange{ - ShardID: "sh2", - DistributionId: distribution, - KeyRangeID: "id2", - LowerBound: []byte("11"), - }) + err = db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh2", + Distribution: distribution, + ID: "id2", + LowerBound: []interface{}{int64(11)}, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, + }).ToDB()) assert.NoError(err) @@ -1247,7 +1435,11 @@ func TestCopySingleShard(t *testing.T) { ShardID: "sh1", ID: "id1", Distribution: distribution, - LowerBound: []byte("1"), + LowerBound: []interface{}{ + int64(1), + }, + + ColumnTypes: []string{qdb.ColumnTypeInteger}, }, }, TargetSessionAttrs: "any", @@ -1287,7 +1479,7 @@ func TestSetStmt(t *testing.T) { ShardID: "sh1", DistributionId: distribution1, KeyRangeID: "id1", - LowerBound: []byte("1"), + LowerBound: [][]byte{[]byte("1")}, }) assert.NoError(err) @@ -1296,7 +1488,7 @@ func TestSetStmt(t *testing.T) { ShardID: "sh2", DistributionId: distribution2, KeyRangeID: "id2", - LowerBound: []byte("1"), + LowerBound: [][]byte{[]byte("1")}, }) assert.NoError(err) @@ -1368,7 +1560,7 @@ func TestMiscRouting(t *testing.T) { ShardID: "sh1", DistributionId: distribution1, KeyRangeID: "id1", - LowerBound: []byte("1"), + LowerBound: [][]byte{[]byte("1")}, }) assert.NoError(err) @@ -1377,7 +1569,7 @@ func TestMiscRouting(t *testing.T) { ShardID: "sh2", DistributionId: distribution2, KeyRangeID: "id2", - LowerBound: []byte("1"), + LowerBound: [][]byte{[]byte("1")}, }) assert.NoError(err) @@ -1440,3 +1632,104 @@ func TestMiscRouting(t *testing.T) { } } } + +func TestHashRouting(t *testing.T) { + assert := assert.New(t) + + type tcase struct { + query string + distribution string + exp routingstate.RoutingState + err error + } + db, _ := qdb.NewMemQDB(MemQDBPath) + distribution1 := "ds1" + + assert.NoError(db.CreateDistribution(context.TODO(), + qdb.NewDistribution(distribution1, + []string{qdb.ColumnTypeVarcharHashed}))) + + err := db.CreateKeyRange(context.TODO(), (&kr.KeyRange{ + ShardID: "sh1", + Distribution: distribution1, + ID: "id1", + LowerBound: kr.KeyRangeBound{ + int64(1), + }, + ColumnTypes: []string{ + qdb.ColumnTypeInteger, + }, + }).ToDB(), + ) + + assert.NoError(err) + + err = db.AlterDistributionAttach(context.TODO(), distribution1, []*qdb.DistributedRelation{ + { + Name: "xx", + DistributionKey: []qdb.DistributionKeyEntry{ + { + Column: "col1", + HashFunction: "murmur", + }, + }, + }, + }) + + assert.NoError(err) + + lc := local.NewLocalCoordinator(db) + + pr, err := qrouter.NewProxyRouter(map[string]*config.Shard{ + "sh1": { + Hosts: nil, + }, + "sh2": { + Hosts: nil, + }, + }, lc, &config.QRouter{ + DefaultRouteBehaviour: "BLOCK", + }) + + assert.NoError(err) + + for _, tt := range []tcase{ + { + query: "INSERT INTO xx (col1) VALUES ('Hello, world!');", + distribution: distribution1, + exp: routingstate.ShardMatchState{ + Route: &routingstate.DataShardRoute{ + Shkey: kr.ShardKey{ + Name: "sh1", + }, + Matchedkr: &kr.KeyRange{ + ID: "id1", + LowerBound: kr.KeyRangeBound{ + uint64(1), + }, + ShardID: "sh1", + Distribution: "ds1", + ColumnTypes: []string{ + qdb.ColumnTypeVarcharHashed, + }, + }, + }, + TargetSessionAttrs: "any", + }, + err: nil, + }, + } { + parserRes, err := lyx.Parse(tt.query) + + assert.NoError(err, "query %s", tt.query) + + tmp, err := pr.Route(context.TODO(), parserRes, session.NewDummyHandler(tt.distribution)) + if tt.err == nil { + assert.NoError(err, "query %s", tt.query) + + assert.Equal(tt.exp, tmp, tt.query) + } else { + assert.Error(tt.err, err, tt.query) + } + } +} diff --git a/router/qrouter/qrouter.go b/router/qrouter/qrouter.go index 29282162d..3fd33abad 100644 --- a/router/qrouter/qrouter.go +++ b/router/qrouter/qrouter.go @@ -3,6 +3,7 @@ package qrouter import ( "context" "fmt" + "github.com/pg-sharding/spqr/pkg/models/kr" "github.com/pg-sharding/spqr/pkg/config" @@ -22,7 +23,7 @@ type QueryRouter interface { WorldShardsRoutes() []*routingstate.DataShardRoute DataShardsRoutes() []*routingstate.DataShardRoute - DeparseKeyWithRangesInternal(ctx context.Context, key string, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) + DeparseKeyWithRangesInternal(ctx context.Context, key []interface{}, krs []*kr.KeyRange) (*routingstate.DataShardRoute, error) Initialized() bool Initialize() bool diff --git a/router/relay/qstate.go b/router/relay/qstate.go index 774c82b2c..a6541d204 100644 --- a/router/relay/qstate.go +++ b/router/relay/qstate.go @@ -41,7 +41,12 @@ func deparseRouteHint(rst RelayStateMgr, params map[string]string) (routehint.Ro return nil, err } - ds, err := rst.QueryRouter().DeparseKeyWithRangesInternal(context.TODO(), val, krs) + // TODO: fix this + compositeKey := []interface{}{ + val, + } + + ds, err := rst.QueryRouter().DeparseKeyWithRangesInternal(context.TODO(), compositeKey, krs) if err != nil { return nil, err } diff --git a/test/drivers/gorm-regress/docker-compose.yaml b/test/drivers/gorm-regress/docker-compose.yaml index ca816e3d8..d169397fe 100644 --- a/test/drivers/gorm-regress/docker-compose.yaml +++ b/test/drivers/gorm-regress/docker-compose.yaml @@ -35,7 +35,7 @@ services: - "6432:6432" environment: - ROUTER_CONFIG=/spqr/test/regress/conf/router.yaml - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log hostname: regress_router container_name: regress_router depends_on: diff --git a/test/drivers/hibernate-regress/docker-compose.yaml b/test/drivers/hibernate-regress/docker-compose.yaml index f216b9b7c..a9a7b4bbd 100644 --- a/test/drivers/hibernate-regress/docker-compose.yaml +++ b/test/drivers/hibernate-regress/docker-compose.yaml @@ -35,7 +35,7 @@ services: - "6432:6432" environment: - ROUTER_CONFIG=/spqr/test/regress/conf/router.yaml - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log hostname: regress_router container_name: regress_router depends_on: diff --git a/test/drivers/jdbc-regress/docker-compose.yaml b/test/drivers/jdbc-regress/docker-compose.yaml index 093723bac..f5319807f 100644 --- a/test/drivers/jdbc-regress/docker-compose.yaml +++ b/test/drivers/jdbc-regress/docker-compose.yaml @@ -35,7 +35,7 @@ services: - "6432:6432" environment: - ROUTER_CONFIG=/spqr/test/regress/conf/router.yaml - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log hostname: regress_router container_name: regress_router depends_on: diff --git a/test/feature/docker-compose.yaml b/test/feature/docker-compose.yaml index 708c34f6c..d4355fc51 100644 --- a/test/feature/docker-compose.yaml +++ b/test/feature/docker-compose.yaml @@ -37,7 +37,7 @@ services: - "7012:7002" environment: - ROUTER_CONFIG=${ROUTER_CONFIG} - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log - 'ROUTER_HOST=host: ''regress_router''' - COORDINATOR_CONFIG=${COORDINATOR_CONFIG} hostname: regress_router @@ -67,7 +67,7 @@ services: - "7022:7002" environment: - ROUTER_CONFIG=${ROUTER_CONFIG} - - ROUTER_LOG='router2.log' + - ROUTER_LOG=/var/log/spqr-router.log - 'ROUTER_HOST=host: ''regress_router_2''' - COORDINATOR_CONFIG=${COORDINATOR_CONFIG_2} hostname: regress_router_2 diff --git a/test/feature/features/move.feature b/test/feature/features/move.feature index 12ca36d3c..6297cc51b 100644 --- a/test/feature/features/move.feature +++ b/test/feature/features/move.feature @@ -179,13 +179,13 @@ Feature: Move test { "Key range ID":"krid3", "Distribution ID":"ds2", - "Lower bound":"a", + "Lower bound":"'a'", "Shard ID":"sh2" }, { "Key range ID":"krid4", "Distribution ID":"ds2", - "Lower bound":"aa", + "Lower bound":"'aa'", "Shard ID":"sh2" } ] diff --git a/test/feature/features/spqrdump.feature b/test/feature/features/spqrdump.feature index 3d1d6fe48..df1222f0d 100644 --- a/test/feature/features/spqrdump.feature +++ b/test/feature/features/spqrdump.feature @@ -40,8 +40,8 @@ Feature: spqr-dump test When I run SQL on host "coordinator" """ CREATE DISTRIBUTION ds1 COLUMN TYPES integer, varchar; - CREATE KEY RANGE krid1 FROM 0 ROUTE TO sh1 FOR DISTRIBUTION ds1; - CREATE KEY RANGE krid2 FROM 11 ROUTE TO sh2 FOR DISTRIBUTION ds1; + CREATE KEY RANGE krid1 FROM 0,'a' ROUTE TO sh1 FOR DISTRIBUTION ds1; + CREATE KEY RANGE krid2 FROM 11,'b' ROUTE TO sh2 FOR DISTRIBUTION ds1; ALTER DISTRIBUTION ds1 ATTACH RELATION test DISTRIBUTION KEY id, id_2; """ Then command return code should be "0" @@ -55,8 +55,8 @@ Feature: spqr-dump test """ CREATE DISTRIBUTION ds1 COLUMN TYPES integer, varchar; ALTER DISTRIBUTION ds1 ATTACH RELATION test DISTRIBUTION KEY id, id_2; - CREATE KEY RANGE krid1 FROM 0 ROUTE TO sh1 FOR DISTRIBUTION ds1; - CREATE KEY RANGE krid2 FROM 11 ROUTE TO sh2 FOR DISTRIBUTION ds1; + CREATE KEY RANGE krid1 FROM 0,'a' ROUTE TO sh1 FOR DISTRIBUTION ds1; + CREATE KEY RANGE krid2 FROM 11,'b' ROUTE TO sh2 FOR DISTRIBUTION ds1; """ Scenario: dump via GRPC works with hashed distribution key diff --git a/test/feature/spqr_test.go b/test/feature/spqr_test.go index 514aaff55..8b6381cd2 100644 --- a/test/feature/spqr_test.go +++ b/test/feature/spqr_test.go @@ -621,7 +621,7 @@ func (tctx *testContext) stepHostIsStarted(service string) error { if err != nil { return fmt.Errorf("failed to get router addr %s: %s", service, err) } - db, err := tctx.connectPostgresql(addr, postgresqlInitialConnectTimeout) + db, err := tctx.connectPostgresql(addr, 10*postgresqlInitialConnectTimeout) if err != nil { return fmt.Errorf("failed to connect to SPQR router %s: %s", service, err) } @@ -1029,6 +1029,12 @@ func InitializeScenario(s *godog.ScenarioContext, t *testing.T, debug bool) { s.Step(`^I run SQL on host "([^"]*)"$`, tctx.stepIRunSQLOnHost) s.Step(`^I execute SQL on host "([^"]*)"$`, tctx.stepIExecuteSql) s.Step(`^SQL result should match (\w+)$`, tctx.stepSQLResultShouldMatch) + s.Step(`^sleep$`, func() error { + + time.Sleep(time.Hour * 10) + + return nil + }) s.Step(`^SQL result should not match (\w+)$`, tctx.stepSQLResultShouldNotMatch) s.Step(`^I record in qdb data transfer transaction with name "([^"]*)"$`, tctx.stepRecordQDBTx) diff --git a/test/regress/docker-compose.yaml b/test/regress/docker-compose.yaml index da69ff3c3..54d0ff249 100644 --- a/test/regress/docker-compose.yaml +++ b/test/regress/docker-compose.yaml @@ -35,7 +35,7 @@ services: - "6432:6432" environment: - ROUTER_CONFIG=/spqr/test/regress/conf/router.yaml - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log hostname: regress_router container_name: regress_router depends_on: diff --git a/test/regress/tests/console/expected/delete_distribution.out b/test/regress/tests/console/expected/delete_distribution.out index 732e6c5c3..112c4f42e 100644 --- a/test/regress/tests/console/expected/delete_distribution.out +++ b/test/regress/tests/console/expected/delete_distribution.out @@ -5,13 +5,13 @@ You can find documentation here https://github.com/pg-sharding/spqr/tree/master/docs -CREATE DISTRIBUTION ds1; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; add distribution ------------------------ distribution id -> ds1 (1 row) -CREATE DISTRIBUTION ds2; +CREATE DISTRIBUTION ds2 COLUMN TYPES int; add distribution ------------------------ distribution id -> ds2 @@ -75,7 +75,7 @@ DROP DISTRIBUTION ds1 CASCADE; SHOW distributions; Distribution ID | Column types -----------------+-------------- - ds2 | + ds2 | integer (1 row) DROP DISTRIBUTION ALL CASCADE; diff --git a/test/regress/tests/console/sql/delete_distribution.sql b/test/regress/tests/console/sql/delete_distribution.sql index 71fb6d57f..015d70ae7 100644 --- a/test/regress/tests/console/sql/delete_distribution.sql +++ b/test/regress/tests/console/sql/delete_distribution.sql @@ -1,5 +1,5 @@ -CREATE DISTRIBUTION ds1; -CREATE DISTRIBUTION ds2; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; +CREATE DISTRIBUTION ds2 COLUMN TYPES int; CREATE KEY RANGE krid1 FROM 1 ROUTE TO sh1 FOR DISTRIBUTION ds1; CREATE KEY RANGE krid2 FROM 11 ROUTE TO sh2 FOR DISTRIBUTION ds1; diff --git a/test/regress/tests/router/expected/alter_distribution.out b/test/regress/tests/router/expected/alter_distribution.out index aad78394c..d03ff20c4 100644 --- a/test/regress/tests/router/expected/alter_distribution.out +++ b/test/regress/tests/router/expected/alter_distribution.out @@ -6,13 +6,13 @@ You can find documentation here https://github.com/pg-sharding/spqr/tree/master/docs -CREATE DISTRIBUTION ds1; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; add distribution ------------------------ distribution id -> ds1 (1 row) -CREATE DISTRIBUTION ds2; +CREATE DISTRIBUTION ds2 COLUMN TYPES int; add distribution ------------------------ distribution id -> ds2 @@ -57,7 +57,7 @@ ALTER DISTRIBUTION ds2 ATTACH RELATION yy DISTRIBUTION KEY w_id; (2 rows) ALTER DISTRIBUTION ds3 ATTACH RELATION xx DISTRIBUTION KEY w_id; -ERROR: no such distribution. +ERROR: distribution "ds3" not found. ALTER DISTRIBUTION ds1 ATTACH RELATION xx DISTRIBUTION KEY w_id; ERROR: relation "xx" is already attached. \c regress diff --git a/test/regress/tests/router/expected/copy_routing.out b/test/regress/tests/router/expected/copy_routing.out index 967db7bf0..89e5df1b7 100644 --- a/test/regress/tests/router/expected/copy_routing.out +++ b/test/regress/tests/router/expected/copy_routing.out @@ -6,7 +6,7 @@ You can find documentation here https://github.com/pg-sharding/spqr/tree/master/docs -CREATE DISTRIBUTION ds1; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; add distribution ------------------------ distribution id -> ds1 diff --git a/test/regress/tests/router/expected/hash_routing.out b/test/regress/tests/router/expected/hash_routing.out index 22960a51d..b169216f7 100644 --- a/test/regress/tests/router/expected/hash_routing.out +++ b/test/regress/tests/router/expected/hash_routing.out @@ -7,7 +7,7 @@ https://github.com/pg-sharding/spqr/tree/master/docs -- SETUP -CREATE DISTRIBUTION ds1 COLUMN TYPES integer; +CREATE DISTRIBUTION ds1 COLUMN TYPES varchar hash; add distribution ------------------------ distribution id -> ds1 @@ -42,7 +42,7 @@ NOTICE: send query to shard(s) : sh2 INSERT INTO xx (col1) VALUES ('test'); NOTICE: send query to shard(s) : sh2 INSERT INTO xx (col1) VALUES ('众口难调'); -NOTICE: send query to shard(s) : sh1 +NOTICE: send query to shard(s) : sh2 INSERT INTO xx (col1) VALUES ('The quick brown fox jumps over the lazy dog'); NOTICE: send query to shard(s) : sh1 INSERT INTO xx (col1) VALUES ('Армия — не просто доброе слово, а очень быстрое дело. Так мы выигрывали все войны. Пока противник рисует карты наступления, мы меняем ландшафты, причём вручную. Когда приходит время атаки, противник теряется на незнакомой местности и приходит в полную небоеготовность. В этом смысл, в этом наша стратегия.'); diff --git a/test/regress/tests/router/expected/mixed_routing.out b/test/regress/tests/router/expected/mixed_routing.out index c6c2e965b..a33851ed3 100644 --- a/test/regress/tests/router/expected/mixed_routing.out +++ b/test/regress/tests/router/expected/mixed_routing.out @@ -153,15 +153,15 @@ CREATE DISTRIBUTION ds2 COLUMN TYPES varchar; (1 row) CREATE KEY RANGE krid3 FROM 00000000-0000-0000-0000-000000000000 ROUTE TO sh1 FOR DISTRIBUTION ds2; - add key range ------------------------------------------------ - bound -> 00000000-0000-0000-0000-000000000000 + add key range +------------------------------------------------- + bound -> '00000000-0000-0000-0000-000000000000' (1 row) CREATE KEY RANGE krid4 FROM 88888888-8888-8888-8888-888888888888 ROUTE TO sh2 FOR DISTRIBUTION ds2; - add key range ------------------------------------------------ - bound -> 88888888-8888-8888-8888-888888888888 + add key range +------------------------------------------------- + bound -> '88888888-8888-8888-8888-888888888888' (1 row) ALTER DISTRIBUTION ds2 ATTACH RELATION xxmixeduuid DISTRIBUTION KEY id; diff --git a/test/regress/tests/router/sql/alter_distribution.sql b/test/regress/tests/router/sql/alter_distribution.sql index de0065851..914dc1923 100644 --- a/test/regress/tests/router/sql/alter_distribution.sql +++ b/test/regress/tests/router/sql/alter_distribution.sql @@ -1,7 +1,7 @@ \c spqr-console -CREATE DISTRIBUTION ds1; -CREATE DISTRIBUTION ds2; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; +CREATE DISTRIBUTION ds2 COLUMN TYPES int; CREATE KEY RANGE krid1 FROM 1 ROUTE TO sh1 FOR DISTRIBUTION ds1; CREATE KEY RANGE krid2 FROM 11 ROUTE TO sh2 FOR DISTRIBUTION ds1; @@ -46,4 +46,5 @@ DROP TABLE yy; \c spqr-console DROP DISTRIBUTION ALL CASCADE; -DROP KEY RANGE ALL; \ No newline at end of file +DROP KEY RANGE ALL; + diff --git a/test/regress/tests/router/sql/copy_routing.sql b/test/regress/tests/router/sql/copy_routing.sql index b57fd9113..8e4010484 100644 --- a/test/regress/tests/router/sql/copy_routing.sql +++ b/test/regress/tests/router/sql/copy_routing.sql @@ -1,5 +1,5 @@ \c spqr-console -CREATE DISTRIBUTION ds1; +CREATE DISTRIBUTION ds1 COLUMN TYPES int; CREATE KEY RANGE krid1 FROM 1 ROUTE TO sh1 FOR DISTRIBUTION ds1; CREATE KEY RANGE krid2 FROM 30 ROUTE TO sh2 FOR DISTRIBUTION ds1; ALTER DISTRIBUTION ds1 ATTACH RELATION copy_test DISTRIBUTION KEY id; diff --git a/test/regress/tests/router/sql/hash_routing.sql b/test/regress/tests/router/sql/hash_routing.sql index 974fb2ff3..4296a43b4 100644 --- a/test/regress/tests/router/sql/hash_routing.sql +++ b/test/regress/tests/router/sql/hash_routing.sql @@ -1,7 +1,7 @@ \c spqr-console -- SETUP -CREATE DISTRIBUTION ds1 COLUMN TYPES integer; +CREATE DISTRIBUTION ds1 COLUMN TYPES varchar hash; CREATE KEY RANGE krid1 FROM 0 ROUTE TO sh1 FOR DISTRIBUTION ds1; CREATE KEY RANGE krid2 FROM 2147483648 ROUTE TO sh2 FOR DISTRIBUTION ds1; diff --git a/test/stress/docker-compose.yaml b/test/stress/docker-compose.yaml index 952875658..2999cf73d 100644 --- a/test/stress/docker-compose.yaml +++ b/test/stress/docker-compose.yaml @@ -27,7 +27,7 @@ services: - "6432:6432" environment: - ROUTER_CONFIG=/spqr/test/stress/router.yaml - - ROUTER_LOG='router1.log' + - ROUTER_LOG=/var/log/spqr-router.log hostname: stress_router container_name: stress_router depends_on: diff --git a/test/xproto/proto_test.go b/test/xproto/proto_test.go index 625871ec5..fc93ef9e8 100644 --- a/test/xproto/proto_test.go +++ b/test/xproto/proto_test.go @@ -2605,3 +2605,152 @@ func TestMultiPortal(t *testing.T) { } } } + +func TestPrepStmtBinaryFormat(t *testing.T) { + conn, err := getC() + if err != nil { + assert.NoError(t, err, "startup failed") + return + } + defer func() { + _ = conn.Close() + }() + + frontend := pgproto3.NewFrontend(conn, conn) + frontend.Send(&pgproto3.StartupMessage{ + ProtocolVersion: 196608, + Parameters: getConnectionParams(), + }) + if err := frontend.Flush(); err != nil { + assert.NoError(t, err, "startup failed") + } + + if err := waitRFQ(frontend); err != nil { + assert.NoError(t, err, "startup failed") + return + } + + type MessageGroup struct { + Request []pgproto3.FrontendMessage + Response []pgproto3.BackendMessage + } + + for _, msgroup := range []MessageGroup{ + { + Request: []pgproto3.FrontendMessage{ + &pgproto3.Query{String: "begin"}, + &pgproto3.Query{String: "insert into t (id) values(1022)"}, + + &pgproto3.Parse{ + Name: "stmtcache_ft_1", + Query: "SELECT * FROM t where id = $1;", + }, + &pgproto3.Describe{ + Name: "stmtcache_ft_1", + ObjectType: 'S', + }, + &pgproto3.Bind{ + PreparedStatement: "stmtcache_ft_1", + Parameters: [][]byte{ + // 1022 + []byte{0, 0, 3, 254}, + }, + ParameterFormatCodes: []int16{1}, + }, + &pgproto3.Execute{}, + &pgproto3.Sync{}, + + &pgproto3.Query{String: "rollback"}, + }, + Response: []pgproto3.BackendMessage{ + + &pgproto3.CommandComplete{ + CommandTag: []byte("BEGIN"), + }, + &pgproto3.ReadyForQuery{ + TxStatus: byte(txstatus.TXACT), + }, + + &pgproto3.CommandComplete{ + CommandTag: []byte("INSERT 0 1"), + }, + + &pgproto3.ReadyForQuery{ + TxStatus: byte(txstatus.TXACT), + }, + + &pgproto3.ParseComplete{}, + + &pgproto3.ParameterDescription{ + ParameterOIDs: []uint32{23}, + }, + + &pgproto3.RowDescription{ + Fields: []pgproto3.FieldDescription{ + { + Name: []byte("id"), + DataTypeOID: 23, + DataTypeSize: 4, + TypeModifier: -1, + TableAttributeNumber: 1, + }, + }, + }, + + &pgproto3.BindComplete{}, + + &pgproto3.DataRow{ + Values: [][]byte{ + []byte("1022"), + }, + }, + + &pgproto3.CommandComplete{ + CommandTag: []byte("SELECT 1"), + }, + + &pgproto3.ReadyForQuery{ + TxStatus: byte(txstatus.TXACT), + }, + + &pgproto3.CommandComplete{ + CommandTag: []byte("ROLLBACK"), + }, + + &pgproto3.ReadyForQuery{ + TxStatus: byte(txstatus.TXIDLE), + }, + }, + }, + } { + for _, msg := range msgroup.Request { + frontend.Send(msg) + } + _ = frontend.Flush() + backendFinished := false + for ind, msg := range msgroup.Response { + if backendFinished { + break + } + retMsg, err := frontend.Receive() + assert.NoError(t, err) + switch retMsgType := retMsg.(type) { + case *pgproto3.RowDescription: + for i := range retMsgType.Fields { + // We don't want to check table OID + retMsgType.Fields[i].TableOID = 0 + } + case *pgproto3.ReadyForQuery: + switch msg.(type) { + case *pgproto3.ReadyForQuery: + break + default: + backendFinished = true + } + default: + break + } + assert.Equal(t, msg, retMsg, fmt.Sprintf("index=%d", ind)) + } + } +} diff --git a/yacc/console/ast.go b/yacc/console/ast.go index ce682320f..61cb5ff3a 100644 --- a/yacc/console/ast.go +++ b/yacc/console/ast.go @@ -87,8 +87,12 @@ type ShardingRuleEntry struct { HashFunction string } +type KeyRangeBound struct { + Pivots [][]byte +} + type KeyRangeDefinition struct { - LowerBound []byte + LowerBound *KeyRangeBound ShardID string KeyRangeID string Distribution string @@ -105,7 +109,7 @@ func (*DistributionDefinition) iCreate() {} func (*ShardingRuleDefinition) iCreate() {} type SplitKeyRange struct { - Border []byte + Border *KeyRangeBound KeyRangeFromID string KeyRangeID string } diff --git a/yacc/console/gram.go b/yacc/console/gram.go index 9543948af..17617b5f1 100644 --- a/yacc/console/gram.go +++ b/yacc/console/gram.go @@ -9,6 +9,7 @@ import __yyfmt__ "fmt" import ( "crypto/rand" + "encoding/binary" "encoding/hex" "strconv" "strings" @@ -22,7 +23,7 @@ func randomHex(n int) (string, error) { return hex.EncodeToString(bytes), nil } -//line gram.y:24 +//line gram.y:25 type yySymType struct { yys int str string @@ -45,6 +46,8 @@ type yySymType struct { lock *Lock unlock *Unlock + krbound *KeyRangeBound + ds *DistributionDefinition kr *KeyRangeDefinition shard *ShardDefinition @@ -248,7 +251,7 @@ const yyEofCode = 1 const yyErrCode = 2 const yyInitialStackSize = 16 -//line gram.y:830 +//line gram.y:845 //line yacctab:1 var yyExca = [...]int8{ @@ -259,84 +262,84 @@ var yyExca = [...]int8{ const yyPrivate = 57344 -const yyLast = 245 +const yyLast = 244 var yyAct = [...]uint8{ - 132, 175, 214, 170, 178, 155, 144, 154, 129, 143, - 139, 116, 101, 92, 171, 172, 173, 56, 141, 97, - 122, 89, 54, 52, 58, 51, 206, 207, 208, 59, - 146, 87, 177, 68, 88, 69, 136, 82, 82, 82, - 82, 120, 106, 82, 81, 147, 210, 85, 209, 200, - 27, 28, 60, 199, 105, 177, 82, 149, 104, 95, - 96, 164, 30, 29, 34, 35, 153, 121, 21, 20, - 24, 25, 26, 31, 32, 146, 91, 107, 108, 36, - 103, 95, 83, 140, 115, 118, 137, 195, 202, 192, - 147, 125, 127, 67, 126, 124, 109, 94, 86, 125, - 80, 123, 98, 90, 33, 133, 134, 135, 128, 119, - 44, 84, 22, 23, 62, 45, 196, 43, 148, 179, - 117, 82, 46, 114, 112, 150, 111, 142, 93, 57, - 198, 197, 151, 185, 184, 77, 76, 42, 166, 160, - 38, 168, 165, 102, 220, 53, 41, 180, 181, 167, - 176, 40, 174, 39, 71, 182, 117, 82, 187, 50, - 193, 186, 183, 70, 72, 188, 79, 190, 49, 130, - 191, 61, 63, 48, 71, 47, 194, 73, 74, 75, - 157, 176, 157, 70, 152, 159, 158, 159, 158, 100, - 82, 162, 65, 201, 37, 189, 204, 203, 163, 1, - 211, 212, 18, 215, 17, 16, 15, 14, 12, 13, - 216, 217, 8, 218, 219, 9, 113, 169, 222, 223, - 221, 215, 138, 224, 110, 78, 19, 205, 145, 213, - 6, 5, 4, 3, 7, 11, 10, 66, 64, 55, - 2, 131, 161, 156, 99, + 132, 177, 213, 152, 180, 172, 144, 151, 150, 156, + 143, 129, 139, 116, 92, 153, 141, 27, 28, 173, + 174, 175, 97, 122, 89, 52, 51, 146, 68, 30, + 29, 34, 35, 69, 179, 21, 20, 24, 25, 26, + 31, 32, 147, 136, 81, 82, 36, 85, 207, 208, + 209, 56, 179, 82, 87, 82, 54, 88, 58, 95, + 96, 82, 82, 59, 120, 106, 210, 201, 82, 166, + 105, 33, 149, 104, 155, 121, 140, 107, 108, 22, + 23, 95, 101, 103, 115, 118, 60, 91, 67, 83, + 137, 125, 127, 146, 197, 198, 193, 80, 203, 125, + 82, 62, 126, 98, 123, 133, 134, 135, 147, 128, + 124, 90, 109, 84, 181, 44, 86, 195, 148, 94, + 45, 119, 43, 117, 114, 157, 57, 46, 53, 142, + 77, 112, 93, 111, 185, 117, 76, 102, 168, 38, + 167, 170, 162, 217, 42, 199, 185, 182, 183, 41, + 178, 169, 40, 176, 61, 63, 82, 186, 184, 157, + 73, 74, 75, 79, 39, 187, 50, 191, 130, 185, + 189, 49, 188, 194, 48, 192, 159, 164, 196, 100, + 82, 161, 160, 178, 165, 71, 47, 71, 65, 190, + 37, 1, 202, 200, 70, 72, 70, 154, 205, 18, + 204, 159, 211, 17, 214, 16, 161, 160, 15, 14, + 12, 215, 13, 216, 8, 9, 113, 219, 214, 218, + 220, 171, 138, 110, 78, 19, 206, 145, 212, 6, + 5, 4, 3, 7, 11, 10, 66, 64, 55, 2, + 131, 163, 158, 99, } var yyPact = [...]int16{ - 44, -1000, 125, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 11, -1000, 124, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 73, 73, -45, -47, -20, 72, 72, 188, 29, 150, - -1000, 72, 72, 72, 114, 113, 56, -1000, -1000, -1000, - -1000, -1000, -1000, 186, 30, 68, 39, -1000, -1000, -1000, - -1000, -30, -50, -1000, 60, -1000, 24, 95, 36, 186, - -54, -1000, 59, -1000, 181, -1000, 129, 129, -1000, -1000, - -1000, -1000, -1000, 1, -4, -17, 186, 35, -1000, 90, - 186, 85, -1000, 117, 52, -18, 12, -51, 129, -1000, - 34, 33, -1000, -1000, 95, -1000, -1000, -1000, 186, -1000, - 153, -1000, -1000, -1000, 186, 186, 186, -26, -1000, -1000, - -1000, 41, 38, -1000, -59, 81, 37, 186, 0, 170, - 11, 150, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 176, - 153, 187, -1000, 5, -1000, -1000, 150, 186, 38, -1000, - 186, -60, 37, -8, -1000, 79, 186, 186, -1000, 170, - 111, 110, -1000, 150, 146, -1000, 153, -1000, -1000, -1000, - 178, 150, -1000, -1000, 150, -1000, -1000, -1000, 45, 148, - -1000, -1000, -1000, -1000, -8, -1000, -1000, 43, -1000, 75, - -1000, -1000, 108, 107, -5, -9, 146, 150, 176, -1000, - -1000, -1000, 46, -60, -1000, 186, -39, -10, -12, 186, - 186, -1000, 186, -1000, -1000, -1000, -1000, -1000, -1000, 186, - 186, -31, -31, 132, -1000, 79, -31, -31, -1000, -1000, - 186, -1000, -1000, -1000, -1000, + 78, 78, -44, -45, 14, 59, 59, 184, 24, 181, + -1000, 59, 59, 59, 114, 108, 53, -1000, -1000, -1000, + -1000, -1000, -1000, 176, 37, 70, 57, -1000, -1000, -1000, + -1000, -7, -47, -1000, 68, -1000, 35, 99, 58, 176, + -51, -1000, 60, -1000, 171, -1000, 123, 123, -1000, -1000, + -1000, -1000, -1000, 16, 12, 6, 176, 51, -1000, 97, + 176, 86, -1000, 96, 64, 5, 20, -48, 123, -1000, + 49, 41, -1000, -1000, 99, -1000, -1000, -1000, 176, -1000, + 152, -1000, -1000, -1000, 176, 176, 176, -19, -1000, -1000, + -1000, 45, 31, -1000, -61, 84, 55, 176, 15, 183, + 19, 181, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 197, + 152, 173, -1000, 13, -1000, -1000, 181, 176, 31, -1000, + 176, -55, 55, -11, -1000, 74, 176, 176, -1000, 183, + 134, -1000, -1000, -1000, -1000, 181, 160, -1000, 152, -1000, + -1000, -1000, 172, 181, -1000, -1000, 183, -1000, -1000, -1000, + 52, 161, -1000, 77, -1000, -1000, -11, -1000, -1000, 50, + -1000, 54, -1000, -1000, 122, 183, 9, 160, 181, 197, + -1000, -1000, 157, 56, -55, -1000, -1000, 176, -17, 8, + -1000, 176, -1000, 176, -1000, -1000, -1000, -1000, -1000, -1000, + 176, -29, 131, -1000, 74, -29, -1000, 176, -1000, -1000, + -1000, } var yyPgo = [...]uint8{ - 0, 244, 8, 243, 242, 241, 5, 0, 12, 240, - 239, 145, 129, 238, 237, 236, 235, 234, 233, 232, - 231, 230, 153, 151, 146, 137, 9, 229, 6, 2, - 11, 228, 4, 227, 1, 226, 225, 224, 222, 10, - 217, 216, 7, 3, 13, 215, 212, 209, 208, 207, - 206, 205, 204, 202, 199, 194, + 0, 243, 11, 7, 8, 242, 241, 240, 3, 0, + 15, 239, 238, 128, 126, 237, 236, 235, 234, 233, + 232, 231, 230, 229, 164, 152, 149, 144, 10, 228, + 6, 2, 13, 227, 4, 226, 1, 225, 224, 223, + 222, 12, 221, 216, 9, 5, 14, 215, 214, 212, + 210, 209, 208, 205, 203, 199, 191, 190, } var yyR1 = [...]int8{ - 0, 54, 55, 55, 9, 9, 9, 9, 9, 9, - 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, - 9, 8, 6, 6, 6, 7, 3, 3, 3, 4, - 4, 5, 2, 2, 2, 1, 1, 13, 14, 44, - 44, 17, 17, 17, 17, 17, 17, 17, 17, 18, - 18, 18, 18, 20, 20, 21, 35, 36, 36, 27, - 27, 29, 39, 38, 38, 37, 19, 19, 19, 19, - 15, 46, 22, 41, 41, 40, 40, 43, 43, 43, - 23, 23, 26, 26, 28, 30, 30, 31, 31, 33, - 33, 33, 32, 32, 34, 24, 24, 24, 24, 25, - 25, 42, 42, 45, 10, 11, 12, 49, 16, 16, - 50, 51, 48, 47, 52, 53, 53, + 0, 56, 57, 57, 11, 11, 11, 11, 11, 11, + 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, + 11, 10, 8, 8, 8, 9, 5, 5, 5, 6, + 6, 7, 2, 2, 2, 1, 1, 15, 16, 46, + 46, 19, 19, 19, 19, 19, 19, 19, 19, 20, + 20, 20, 20, 22, 22, 23, 37, 38, 38, 29, + 29, 31, 41, 40, 40, 39, 21, 21, 21, 21, + 17, 48, 24, 43, 43, 42, 42, 45, 45, 45, + 45, 25, 25, 28, 28, 30, 32, 32, 33, 33, + 35, 35, 35, 34, 34, 36, 3, 3, 4, 4, + 26, 26, 27, 27, 44, 44, 47, 12, 13, 14, + 51, 18, 18, 52, 53, 50, 49, 54, 55, 55, } var yyR2 = [...]int8{ @@ -347,63 +350,63 @@ var yyR2 = [...]int8{ 0, 2, 4, 2, 4, 3, 4, 3, 3, 2, 2, 2, 2, 4, 4, 3, 2, 2, 4, 3, 1, 2, 5, 1, 2, 2, 2, 2, 2, 2, - 3, 2, 3, 3, 0, 3, 1, 1, 1, 1, - 6, 5, 1, 2, 2, 2, 0, 2, 2, 1, - 1, 1, 3, 0, 3, 9, 9, 8, 8, 5, - 4, 1, 3, 2, 3, 3, 2, 6, 3, 3, - 4, 4, 2, 1, 5, 3, 3, + 3, 2, 3, 3, 0, 3, 1, 1, 2, 1, + 1, 6, 5, 1, 2, 2, 2, 0, 2, 2, + 1, 1, 1, 3, 0, 3, 1, 1, 1, 3, + 9, 8, 5, 4, 1, 3, 2, 3, 3, 2, + 6, 3, 3, 4, 4, 2, 1, 5, 3, 3, } var yyChk = [...]int16{ - -1000, -54, -9, -18, -19, -20, -21, -17, -46, -45, - -15, -16, -48, -47, -49, -50, -51, -52, -53, -35, + -1000, -56, -11, -20, -21, -22, -23, -19, -48, -47, + -17, -18, -50, -49, -51, -52, -53, -54, -55, -37, 25, 24, 68, 69, 26, 27, 28, 6, 7, 19, - 18, 29, 30, 60, 20, 21, 35, -55, 15, -22, - -23, -24, -25, 44, 37, 42, 49, -22, -23, -24, - -25, 70, 70, -11, 42, -10, 37, -12, 44, 49, - 72, -11, 42, -11, -13, 4, -14, 64, 4, -6, - 13, 4, 14, -11, -11, -11, 22, 22, -36, -12, - 44, -7, 4, 52, 43, -7, 59, 61, 64, 71, - 43, 52, -44, 33, 61, -7, -7, 73, 43, -1, - 8, -8, 14, -8, 57, 58, 59, -7, -7, 61, - -37, 36, 34, -41, 38, -7, -30, 39, -7, 57, - 59, 55, 71, -8, 61, -7, 61, -7, -44, -2, - 16, -5, -7, -7, -7, -7, 62, 45, -38, -39, - 45, 77, -30, -26, -28, -31, 38, 53, -7, 57, - -6, -8, 14, 55, -42, -6, -3, 4, 10, 9, - -2, -4, 4, 11, 56, -6, -7, -39, -7, -40, - -43, 74, 75, 76, -26, -34, -28, 63, -32, 40, - -7, -7, -6, -8, 23, 23, -42, 12, -2, 17, - -6, -6, 44, 12, -34, 44, 41, 23, 23, 58, - 58, -6, 42, -43, -7, -33, 65, 66, 67, 58, - 58, -7, -7, -27, -29, -7, -7, -7, -34, -34, - 12, -32, -34, -34, -29, + 18, 29, 30, 60, 20, 21, 35, -57, 15, -24, + -25, -26, -27, 44, 37, 42, 49, -24, -25, -26, + -27, 70, 70, -13, 42, -12, 37, -14, 44, 49, + 72, -13, 42, -13, -15, 4, -16, 64, 4, -8, + 13, 4, 14, -13, -13, -13, 22, 22, -38, -14, + 44, -9, 4, 52, 43, -9, 59, 61, 64, 71, + 43, 52, -46, 33, 61, -9, -9, 73, 43, -1, + 8, -10, 14, -10, 57, 58, 59, -9, -9, 61, + -39, 36, 34, -43, 38, -9, -32, 39, -9, 57, + 59, 55, 71, -10, 61, -9, 61, -9, -46, -2, + 16, -7, -9, -9, -9, -9, 62, 45, -40, -41, + 45, 77, -32, -28, -30, -33, 38, 53, -9, 57, + -4, -3, -8, -10, 14, 55, -44, -8, -5, 4, + 10, 9, -2, -6, 4, 11, 56, -8, -9, -41, + -9, -42, -45, 74, 75, 76, -28, -36, -30, 63, + -34, 40, -9, -9, -4, 12, 23, -44, 12, -2, + 17, -8, -4, 44, 12, 40, -36, 44, 41, 23, + -3, 58, -8, 42, -45, -9, -35, 65, 66, 67, + 58, -9, -29, -31, -9, -9, -36, 12, -34, -36, + -31, } var yyDef = [...]int8{ 0, -2, 2, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 113, 0, 0, 0, 0, 0, 0, 1, 3, 49, + 116, 0, 0, 0, 0, 0, 0, 1, 3, 49, 50, 51, 52, 0, 0, 0, 0, 66, 67, 68, 69, 0, 0, 41, 0, 43, 0, 40, 0, 0, - 0, 71, 0, 103, 35, 37, 0, 0, 38, 112, + 0, 71, 0, 106, 35, 37, 0, 0, 38, 115, 22, 23, 24, 0, 0, 0, 0, 0, 56, 0, - 0, 74, 25, 86, 0, 0, 0, 0, 0, 55, - 0, 0, 45, 39, 40, 106, 47, 48, 0, 70, - 0, 108, 21, 109, 0, 0, 0, 0, 115, 116, - 57, 0, 0, 72, 0, 86, 0, 0, 0, 0, - 0, 0, 53, 54, 42, 105, 44, 104, 46, 36, - 0, 0, 31, 0, 110, 111, 0, 0, 65, 63, - 0, 0, 0, 0, 82, 93, 0, 0, 85, 0, - 0, 0, 21, 0, 100, 101, 0, 26, 27, 28, - 0, 0, 29, 30, 0, 114, 58, 64, 0, 73, - 76, 77, 78, 79, 0, 81, 83, 0, 84, 0, - 87, 88, 0, 0, 0, 0, 99, 0, 34, 32, - 33, 107, 0, 0, 80, 0, 0, 0, 0, 0, - 0, 102, 0, 75, 94, 92, 89, 90, 91, 0, - 0, 0, 0, 62, 60, 93, 0, 0, 97, 98, - 0, 61, 95, 96, 59, + 0, 74, 25, 87, 0, 0, 0, 0, 0, 55, + 0, 0, 45, 39, 40, 109, 47, 48, 0, 70, + 0, 111, 21, 112, 0, 0, 0, 0, 118, 119, + 57, 0, 0, 72, 0, 87, 0, 0, 0, 0, + 0, 0, 53, 54, 42, 108, 44, 107, 46, 36, + 0, 0, 31, 0, 113, 114, 0, 0, 65, 63, + 0, 0, 0, 0, 83, 94, 0, 0, 86, 0, + 0, 98, 96, 97, 21, 0, 103, 104, 0, 26, + 27, 28, 0, 0, 29, 30, 0, 117, 58, 64, + 0, 73, 76, 77, 79, 80, 0, 82, 84, 0, + 85, 0, 88, 89, 0, 0, 0, 102, 0, 34, + 32, 33, 110, 0, 0, 78, 81, 0, 0, 0, + 99, 0, 105, 0, 75, 95, 93, 90, 91, 92, + 0, 0, 62, 60, 94, 0, 101, 0, 61, 100, + 59, } var yyTok1 = [...]int8{ @@ -764,179 +767,179 @@ yydefault: case 2: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:212 +//line gram.y:222 { } case 3: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:213 +//line gram.y:223 { } case 4: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:218 +//line gram.y:228 { setParseTree(yylex, yyDollar[1].create) } case 5: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:222 +//line gram.y:232 { setParseTree(yylex, yyDollar[1].create) } case 6: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:226 +//line gram.y:236 { setParseTree(yylex, yyDollar[1].trace) } case 7: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:230 +//line gram.y:240 { setParseTree(yylex, yyDollar[1].stoptrace) } case 8: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:234 +//line gram.y:244 { setParseTree(yylex, yyDollar[1].drop) } case 9: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:238 +//line gram.y:248 { setParseTree(yylex, yyDollar[1].lock) } case 10: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:242 +//line gram.y:252 { setParseTree(yylex, yyDollar[1].unlock) } case 11: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:246 +//line gram.y:256 { setParseTree(yylex, yyDollar[1].show) } case 12: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:250 +//line gram.y:260 { setParseTree(yylex, yyDollar[1].kill) } case 13: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:254 +//line gram.y:264 { setParseTree(yylex, yyDollar[1].listen) } case 14: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:258 +//line gram.y:268 { setParseTree(yylex, yyDollar[1].shutdown) } case 15: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:262 +//line gram.y:272 { setParseTree(yylex, yyDollar[1].split) } case 16: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:266 +//line gram.y:276 { setParseTree(yylex, yyDollar[1].move) } case 17: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:270 +//line gram.y:280 { setParseTree(yylex, yyDollar[1].unite) } case 18: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:274 +//line gram.y:284 { setParseTree(yylex, yyDollar[1].register_router) } case 19: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:278 +//line gram.y:288 { setParseTree(yylex, yyDollar[1].unregister_router) } case 20: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:282 +//line gram.y:292 { setParseTree(yylex, yyDollar[1].alter) } case 21: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:287 +//line gram.y:297 { yyVAL.uinteger = uint(yyDollar[1].uinteger) } case 22: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:292 +//line gram.y:302 { yyVAL.str = string(yyDollar[1].str) } case 23: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:296 +//line gram.y:306 { yyVAL.str = string(yyDollar[1].str) } case 24: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:298 +//line gram.y:308 { yyVAL.str = strconv.Itoa(int(yyDollar[1].uinteger)) } case 25: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:303 +//line gram.y:313 { yyVAL.str = string(yyDollar[1].str) } case 26: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:309 +//line gram.y:319 { yyVAL.str = yyDollar[1].str } case 27: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:311 +//line gram.y:321 { yyVAL.str = "AND" } case 28: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:313 +//line gram.y:323 { yyVAL.str = "OR" } case 29: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:318 +//line gram.y:328 { yyVAL.str = yyDollar[1].str } case 30: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:320 +//line gram.y:330 { yyVAL.str = "=" } case 31: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:326 +//line gram.y:336 { yyVAL.colref = ColumnRef{ ColName: yyDollar[1].str, @@ -944,13 +947,13 @@ yydefault: } case 32: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:334 +//line gram.y:344 { yyVAL.where = yyDollar[2].where } case 33: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:337 +//line gram.y:347 { yyVAL.where = WhereClauseLeaf{ ColRef: yyDollar[1].colref, @@ -960,7 +963,7 @@ yydefault: } case 34: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:345 +//line gram.y:355 { yyVAL.where = WhereClauseOp{ Op: yyDollar[2].str, @@ -970,19 +973,19 @@ yydefault: } case 35: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:355 +//line gram.y:365 { yyVAL.where = WhereClauseEmpty{} } case 36: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:359 +//line gram.y:369 { yyVAL.where = yyDollar[2].where } case 37: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:366 +//line gram.y:376 { switch v := strings.ToLower(string(yyDollar[1].str)); v { case DatabasesStr, RoutersStr, PoolsStr, ShardsStr, BackendConnectionsStr, KeyRangesStr, ShardingRules, ClientsStr, StatusStr, DistributionsStr, VersionStr, RelationsStr, TaskGroupStr, PreparedStatementsStr: @@ -993,7 +996,7 @@ yydefault: } case 38: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:377 +//line gram.y:387 { switch v := string(yyDollar[1].str); v { case ClientStr: @@ -1004,97 +1007,97 @@ yydefault: } case 39: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:387 +//line gram.y:397 { yyVAL.bool = true } case 40: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:387 +//line gram.y:397 { yyVAL.bool = false } case 41: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:391 +//line gram.y:401 { yyVAL.drop = &Drop{Element: yyDollar[2].key_range_selector} } case 42: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:395 +//line gram.y:405 { yyVAL.drop = &Drop{Element: &KeyRangeSelector{KeyRangeID: `*`}} } case 43: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:399 +//line gram.y:409 { yyVAL.drop = &Drop{Element: yyDollar[2].sharding_rule_selector} } case 44: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:403 +//line gram.y:413 { yyVAL.drop = &Drop{Element: &ShardingRuleSelector{ID: `*`}} } case 45: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:407 +//line gram.y:417 { yyVAL.drop = &Drop{Element: yyDollar[2].distribution_selector, CascadeDelete: yyDollar[3].bool} } case 46: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:411 +//line gram.y:421 { yyVAL.drop = &Drop{Element: &DistributionSelector{ID: `*`}, CascadeDelete: yyDollar[4].bool} } case 47: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:415 +//line gram.y:425 { yyVAL.drop = &Drop{Element: &ShardSelector{ID: yyDollar[3].str}} } case 48: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:419 +//line gram.y:429 { yyVAL.drop = &Drop{Element: &TaskGroupSelector{}} } case 49: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:426 +//line gram.y:436 { yyVAL.create = &Create{Element: yyDollar[2].ds} } case 50: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:431 +//line gram.y:441 { yyVAL.create = &Create{Element: yyDollar[2].sharding_rule} } case 51: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:436 +//line gram.y:446 { yyVAL.create = &Create{Element: yyDollar[2].kr} } case 52: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:440 +//line gram.y:450 { yyVAL.create = &Create{Element: yyDollar[2].shard} } case 53: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:446 +//line gram.y:456 { yyVAL.trace = &TraceStmt{All: true} } case 54: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:449 +//line gram.y:459 { yyVAL.trace = &TraceStmt{ Client: yyDollar[4].uinteger, @@ -1102,19 +1105,19 @@ yydefault: } case 55: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:457 +//line gram.y:467 { yyVAL.stoptrace = &StopTraceStmt{} } case 56: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:463 +//line gram.y:473 { yyVAL.alter = &Alter{Element: yyDollar[2].alter_distribution} } case 57: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:469 +//line gram.y:479 { yyVAL.alter_distribution = &AlterDistribution{ Element: &AttachRelation{ @@ -1125,7 +1128,7 @@ yydefault: } case 58: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:478 +//line gram.y:488 { yyVAL.alter_distribution = &AlterDistribution{ Element: &DetachRelation{ @@ -1136,13 +1139,13 @@ yydefault: } case 59: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:490 +//line gram.y:500 { yyVAL.dEntrieslist = append(yyDollar[1].dEntrieslist, yyDollar[3].distrKeyEntry) } case 60: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:492 +//line gram.y:502 { yyVAL.dEntrieslist = []DistributionKeyEntry{ yyDollar[1].distrKeyEntry, @@ -1150,7 +1153,7 @@ yydefault: } case 61: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:502 +//line gram.y:512 { yyVAL.distrKeyEntry = DistributionKeyEntry{ Column: yyDollar[1].str, @@ -1159,7 +1162,7 @@ yydefault: } case 62: yyDollar = yyS[yypt-5 : yypt+1] -//line gram.y:511 +//line gram.y:521 { yyVAL.distributed_relation = &DistributedRelation{ Name: yyDollar[2].str, @@ -1168,61 +1171,61 @@ yydefault: } case 63: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:520 +//line gram.y:530 { yyVAL.relations = []*DistributedRelation{yyDollar[1].distributed_relation} } case 64: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:522 +//line gram.y:532 { yyVAL.relations = append(yyDollar[1].relations, yyDollar[2].distributed_relation) } case 65: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:527 +//line gram.y:537 { yyVAL.relations = yyDollar[2].relations } case 66: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:533 +//line gram.y:543 { yyVAL.create = &Create{Element: yyDollar[2].ds} } case 67: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:538 +//line gram.y:548 { yyVAL.create = &Create{Element: yyDollar[2].sharding_rule} } case 68: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:543 +//line gram.y:553 { yyVAL.create = &Create{Element: yyDollar[2].kr} } case 69: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:547 +//line gram.y:557 { yyVAL.create = &Create{Element: yyDollar[2].shard} } case 70: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:554 +//line gram.y:564 { yyVAL.show = &Show{Cmd: yyDollar[2].str, Where: yyDollar[3].where} } case 71: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:560 +//line gram.y:570 { yyVAL.lock = &Lock{KeyRangeID: yyDollar[2].key_range_selector.KeyRangeID} } case 72: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:568 +//line gram.y:578 { yyVAL.ds = &DistributionDefinition{ ID: yyDollar[2].str, @@ -1231,26 +1234,26 @@ yydefault: } case 73: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:576 +//line gram.y:586 { yyVAL.strlist = yyDollar[3].strlist } case 74: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:578 +//line gram.y:588 { /* empty column types should be prohibited */ yyVAL.strlist = nil } case 75: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:584 +//line gram.y:594 { yyVAL.strlist = append(yyDollar[1].strlist, yyDollar[3].str) } case 76: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:586 +//line gram.y:596 { yyVAL.strlist = []string{ yyDollar[1].str, @@ -1258,31 +1261,37 @@ yydefault: } case 77: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:593 +//line gram.y:603 { yyVAL.str = "varchar" } case 78: - yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:595 + yyDollar = yyS[yypt-2 : yypt+1] +//line gram.y:605 { - yyVAL.str = "integer" + yyVAL.str = "varchar hashed" } case 79: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:597 +//line gram.y:607 { yyVAL.str = "integer" } case 80: + yyDollar = yyS[yypt-1 : yypt+1] +//line gram.y:609 + { + yyVAL.str = "integer" + } + case 81: yyDollar = yyS[yypt-6 : yypt+1] -//line gram.y:603 +//line gram.y:615 { yyVAL.sharding_rule = &ShardingRuleDefinition{ID: yyDollar[3].str, TableName: yyDollar[4].str, Entries: yyDollar[5].entrieslist, Distribution: yyDollar[6].str} } - case 81: + case 82: yyDollar = yyS[yypt-5 : yypt+1] -//line gram.y:608 +//line gram.y:620 { str, err := randomHex(6) if err != nil { @@ -1290,149 +1299,155 @@ yydefault: } yyVAL.sharding_rule = &ShardingRuleDefinition{ID: "shrule" + str, TableName: yyDollar[3].str, Entries: yyDollar[4].entrieslist, Distribution: yyDollar[5].str} } - case 82: + case 83: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:617 +//line gram.y:629 { yyVAL.entrieslist = make([]ShardingRuleEntry, 0) yyVAL.entrieslist = append(yyVAL.entrieslist, yyDollar[1].shruleEntry) } - case 83: + case 84: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:623 +//line gram.y:635 { yyVAL.entrieslist = append(yyDollar[1].entrieslist, yyDollar[2].shruleEntry) } - case 84: + case 85: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:629 +//line gram.y:641 { yyVAL.shruleEntry = ShardingRuleEntry{ Column: yyDollar[1].str, HashFunction: yyDollar[2].str, } } - case 85: + case 86: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:638 +//line gram.y:650 { yyVAL.str = yyDollar[2].str } - case 86: + case 87: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:641 +//line gram.y:653 { yyVAL.str = "" } - case 87: + case 88: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:645 +//line gram.y:657 { yyVAL.str = yyDollar[2].str } - case 88: + case 89: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:650 +//line gram.y:662 { yyVAL.str = yyDollar[2].str } - case 89: + case 90: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:656 +//line gram.y:668 { yyVAL.str = "identity" } - case 90: + case 91: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:658 +//line gram.y:670 { yyVAL.str = "murmur" } - case 91: + case 92: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:660 +//line gram.y:672 { yyVAL.str = "city" } - case 92: + case 93: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:666 +//line gram.y:678 { yyVAL.str = yyDollar[3].str } - case 93: + case 94: yyDollar = yyS[yypt-0 : yypt+1] -//line gram.y:668 +//line gram.y:680 { yyVAL.str = "" } - case 94: + case 95: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:673 +//line gram.y:685 { yyVAL.str = yyDollar[3].str } - case 95: - yyDollar = yyS[yypt-9 : yypt+1] -//line gram.y:679 + case 96: + yyDollar = yyS[yypt-1 : yypt+1] +//line gram.y:690 { - yyVAL.kr = &KeyRangeDefinition{ - KeyRangeID: yyDollar[3].str, - LowerBound: []byte(yyDollar[5].str), - ShardID: yyDollar[8].str, - Distribution: yyDollar[9].str, + yyVAL.bytes = []byte(yyDollar[1].str) + } + case 97: + yyDollar = yyS[yypt-1 : yypt+1] +//line gram.y:693 + { + buf := make([]byte, 8) + binary.PutVarint(buf, int64(yyDollar[1].uinteger)) + yyVAL.bytes = buf + } + case 98: + yyDollar = yyS[yypt-1 : yypt+1] +//line gram.y:700 + { + yyVAL.krbound = &KeyRangeBound{ + Pivots: [][]byte{ + yyDollar[1].bytes, + }, } } - case 96: + case 99: + yyDollar = yyS[yypt-3 : yypt+1] +//line gram.y:707 + { + yyVAL.krbound = &KeyRangeBound{ + Pivots: append(yyDollar[1].krbound.Pivots, yyDollar[3].bytes), + } + } + case 100: yyDollar = yyS[yypt-9 : yypt+1] -//line gram.y:688 +//line gram.y:716 { yyVAL.kr = &KeyRangeDefinition{ KeyRangeID: yyDollar[3].str, - LowerBound: []byte(strconv.FormatUint(uint64(yyDollar[5].uinteger), 10)), + LowerBound: yyDollar[5].krbound, ShardID: yyDollar[8].str, Distribution: yyDollar[9].str, } } - case 97: - yyDollar = yyS[yypt-8 : yypt+1] -//line gram.y:697 - { - str, err := randomHex(6) - if err != nil { - panic(err) - } - yyVAL.kr = &KeyRangeDefinition{ - LowerBound: []byte(yyDollar[4].str), - Distribution: yyDollar[6].str, - ShardID: yyDollar[7].str, - KeyRangeID: "kr" + str, - } - } - case 98: + case 101: yyDollar = yyS[yypt-8 : yypt+1] -//line gram.y:710 +//line gram.y:725 { str, err := randomHex(6) if err != nil { panic(err) } yyVAL.kr = &KeyRangeDefinition{ - LowerBound: []byte(strconv.FormatUint(uint64(yyDollar[4].uinteger), 10)), + LowerBound: yyDollar[4].krbound, ShardID: yyDollar[7].str, - KeyRangeID: "kr" + str, Distribution: yyDollar[8].str, + KeyRangeID: "kr" + str, } } - case 99: + case 102: yyDollar = yyS[yypt-5 : yypt+1] -//line gram.y:725 +//line gram.y:740 { yyVAL.shard = &ShardDefinition{Id: yyDollar[2].str, Hosts: yyDollar[5].strlist} } - case 100: + case 103: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:730 +//line gram.y:745 { str, err := randomHex(6) if err != nil { @@ -1440,99 +1455,99 @@ yydefault: } yyVAL.shard = &ShardDefinition{Id: "shard" + str, Hosts: yyDollar[4].strlist} } - case 101: + case 104: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:740 +//line gram.y:755 { yyVAL.strlist = []string{yyDollar[1].str} } - case 102: + case 105: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:745 +//line gram.y:760 { yyVAL.strlist = append(yyDollar[1].strlist, yyDollar[3].str) } - case 103: + case 106: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:751 +//line gram.y:766 { yyVAL.unlock = &Unlock{KeyRangeID: yyDollar[2].key_range_selector.KeyRangeID} } - case 104: + case 107: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:757 +//line gram.y:772 { yyVAL.sharding_rule_selector = &ShardingRuleSelector{ID: yyDollar[3].str} } - case 105: + case 108: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:763 +//line gram.y:778 { yyVAL.key_range_selector = &KeyRangeSelector{KeyRangeID: yyDollar[3].str} } - case 106: + case 109: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:769 +//line gram.y:784 { yyVAL.distribution_selector = &DistributionSelector{ID: yyDollar[2].str} } - case 107: + case 110: yyDollar = yyS[yypt-6 : yypt+1] -//line gram.y:775 +//line gram.y:790 { - yyVAL.split = &SplitKeyRange{KeyRangeID: yyDollar[2].key_range_selector.KeyRangeID, KeyRangeFromID: yyDollar[4].str, Border: []byte(yyDollar[6].str)} + yyVAL.split = &SplitKeyRange{KeyRangeID: yyDollar[2].key_range_selector.KeyRangeID, KeyRangeFromID: yyDollar[4].str, Border: yyDollar[6].krbound} } - case 108: + case 111: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:781 +//line gram.y:796 { yyVAL.kill = &Kill{Cmd: yyDollar[2].str, Target: yyDollar[3].uinteger} } - case 109: + case 112: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:784 +//line gram.y:799 { yyVAL.kill = &Kill{Cmd: "client", Target: yyDollar[3].uinteger} } - case 110: + case 113: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:790 +//line gram.y:805 { yyVAL.move = &MoveKeyRange{KeyRangeID: yyDollar[2].key_range_selector.KeyRangeID, DestShardID: yyDollar[4].str} } - case 111: + case 114: yyDollar = yyS[yypt-4 : yypt+1] -//line gram.y:796 +//line gram.y:811 { yyVAL.unite = &UniteKeyRange{KeyRangeIDL: yyDollar[2].key_range_selector.KeyRangeID, KeyRangeIDR: yyDollar[4].str} } - case 112: + case 115: yyDollar = yyS[yypt-2 : yypt+1] -//line gram.y:802 +//line gram.y:817 { yyVAL.listen = &Listen{addr: yyDollar[2].str} } - case 113: + case 116: yyDollar = yyS[yypt-1 : yypt+1] -//line gram.y:808 +//line gram.y:823 { yyVAL.shutdown = &Shutdown{} } - case 114: + case 117: yyDollar = yyS[yypt-5 : yypt+1] -//line gram.y:816 +//line gram.y:831 { yyVAL.register_router = &RegisterRouter{ID: yyDollar[3].str, Addr: yyDollar[5].str} } - case 115: + case 118: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:822 +//line gram.y:837 { yyVAL.unregister_router = &UnregisterRouter{ID: yyDollar[3].str} } - case 116: + case 119: yyDollar = yyS[yypt-3 : yypt+1] -//line gram.y:827 +//line gram.y:842 { yyVAL.unregister_router = &UnregisterRouter{ID: `*`} } diff --git a/yacc/console/gram.y b/yacc/console/gram.y index 4f1e45fb8..edd7c5239 100644 --- a/yacc/console/gram.y +++ b/yacc/console/gram.y @@ -5,6 +5,7 @@ package spqrparser import ( "crypto/rand" "encoding/hex" + "encoding/binary" "strings" "strconv" ) @@ -42,6 +43,10 @@ func randomHex(n int) (string, error) { lock *Lock unlock *Unlock + + krbound *KeyRangeBound + + ds *DistributionDefinition kr *KeyRangeDefinition shard *ShardDefinition @@ -107,6 +112,11 @@ func randomHex(n int) (string, error) { %token ICONST + +%type key_range_bound_elem + +%type key_range_bound + // ';' %token TSEMICOLON @@ -592,6 +602,8 @@ col_types_list: col_types_elem: VARCHAR { $$ = "varchar" + } | VARCHAR HASH { + $$ = "varchar hashed" } | INTEGER { $$ = "integer" } | INT { @@ -674,49 +686,52 @@ distribution_membership: $$ = $3 } -key_range_define_stmt: - KEY RANGE any_id FROM any_val ROUTE TO any_id distribution_membership - { - $$ = &KeyRangeDefinition{ - KeyRangeID: $3, - LowerBound: []byte($5), - ShardID: $8, - Distribution: $9, +key_range_bound_elem: + any_val { + $$ = []byte($1) + } + | any_uint { + buf := make([]byte, 8) + binary.PutVarint(buf, int64($1)) + $$ = buf + } + +key_range_bound: + key_range_bound_elem { + $$ = &KeyRangeBound{ + Pivots: [][]byte{ + $1, + }, + } + } + | key_range_bound TCOMMA key_range_bound_elem { + $$ = &KeyRangeBound{ + Pivots: append($1.Pivots, $3), } } - | KEY RANGE any_id FROM any_uint ROUTE TO any_id distribution_membership + + +key_range_define_stmt: + KEY RANGE any_id FROM key_range_bound ROUTE TO any_id distribution_membership { $$ = &KeyRangeDefinition{ KeyRangeID: $3, - LowerBound: []byte(strconv.FormatUint(uint64($5), 10)), + LowerBound: $5, ShardID: $8, Distribution: $9, } } - | KEY RANGE FROM any_val ROUTE TO any_id distribution_membership - { - str, err := randomHex(6) - if err != nil { - panic(err) - } - $$ = &KeyRangeDefinition{ - LowerBound: []byte($4), - Distribution: $6, - ShardID: $7, - KeyRangeID: "kr"+str, - } - } - | KEY RANGE FROM any_uint ROUTE TO any_id distribution_membership + | KEY RANGE FROM key_range_bound ROUTE TO any_id distribution_membership { str, err := randomHex(6) if err != nil { panic(err) } $$ = &KeyRangeDefinition{ - LowerBound: []byte(strconv.FormatUint(uint64($4), 10)), + LowerBound: $4, ShardID: $7, - KeyRangeID: "kr"+str, Distribution: $8, + KeyRangeID: "kr"+str, } } @@ -771,9 +786,9 @@ distribution_select_stmt: } split_key_range_stmt: - SPLIT key_range_stmt FROM any_id BY any_val + SPLIT key_range_stmt FROM any_id BY key_range_bound { - $$ = &SplitKeyRange{KeyRangeID: $2.KeyRangeID, KeyRangeFromID: $4, Border: []byte($6)} + $$ = &SplitKeyRange{KeyRangeID: $2.KeyRangeID, KeyRangeFromID: $4, Border: $6} } kill_stmt: diff --git a/yacc/console/yx_test.go b/yacc/console/yx_test.go index 30f40dcdc..ef03ee7f9 100644 --- a/yacc/console/yx_test.go +++ b/yacc/console/yx_test.go @@ -220,7 +220,11 @@ func TestKeyRange(t *testing.T) { ShardID: "sh1", KeyRangeID: "krid1", Distribution: "ds1", - LowerBound: []byte("1"), + LowerBound: &spqrparser.KeyRangeBound{ + Pivots: [][]byte{ + []byte{2, 0, 0, 0, 0, 0, 0, 0}, + }, + }, }, }, err: nil, @@ -233,7 +237,31 @@ func TestKeyRange(t *testing.T) { ShardID: "sh2", KeyRangeID: "krid2", Distribution: "ds1", - LowerBound: []byte("88888888-8888-8888-8888-888888888889"), + LowerBound: &spqrparser.KeyRangeBound{ + Pivots: [][]byte{ + []byte("88888888-8888-8888-8888-888888888889"), + }, + }, + }, + }, + err: nil, + }, + + { + query: ` + CREATE KEY RANGE krid1 FROM 0, 'a' ROUTE TO sh1 FOR DISTRIBUTION ds1;`, + + exp: &spqrparser.Create{ + Element: &spqrparser.KeyRangeDefinition{ + ShardID: "sh1", + KeyRangeID: "krid1", + Distribution: "ds1", + LowerBound: &spqrparser.KeyRangeBound{ + Pivots: [][]byte{ + []byte{0, 0, 0, 0, 0, 0, 0, 0}, + []byte("a"), + }, + }, }, }, err: nil, @@ -309,7 +337,11 @@ func TestSplitKeyRange(t *testing.T) { { query: "SPLIT KEY RANGE krid3 FROM krid1 BY 5;", exp: &spqrparser.SplitKeyRange{ - Border: []byte("5"), + Border: &spqrparser.KeyRangeBound{ + Pivots: [][]byte{ + {10, 0, 0, 0, 0, 0, 0, 0}, + }, + }, KeyRangeFromID: "krid1", KeyRangeID: "krid3", }, @@ -505,6 +537,18 @@ func TestDistribution(t *testing.T) { }, err: nil, }, + { + query: "CREATE DISTRIBUTION db1 COLUMN TYPES varchar hash;", + exp: &spqrparser.Create{ + Element: &spqrparser.DistributionDefinition{ + ID: "db1", + ColTypes: []string{ + "varchar hashed", + }, + }, + }, + err: nil, + }, { query: "CREATE DISTRIBUTION db1 COLUMN TYPES varchar, varchar;", exp: &spqrparser.Create{