From eddd33cae2afda2db10f39e7f107366d8df87236 Mon Sep 17 00:00:00 2001 From: shivaji-dgraph Date: Tue, 13 May 2025 18:04:33 +0530 Subject: [PATCH 1/4] Add support for HA and multishard functionality in import APIs --- dgraph/cmd/dgraphimport/import_client.go | 71 +- dgraph/cmd/dgraphimport/import_test.go | 275 ++++++- dgraphapi/cluster.go | 45 +- dgraphtest/local_cluster.go | 9 + edgraph/server.go | 34 +- go.mod | 2 +- go.sum | 4 +- protos/pb.proto | 10 +- protos/pb/pb.pb.go | 911 +++++++++++------------ protos/pb/pb_grpc.pb.go | 78 +- worker/draft.go | 66 +- worker/import.go | 513 ++++++++++--- x/health.go | 17 +- 13 files changed, 1324 insertions(+), 711 deletions(-) diff --git a/dgraph/cmd/dgraphimport/import_client.go b/dgraph/cmd/dgraphimport/import_client.go index c7a79b79e02..5e6a56a7dad 100644 --- a/dgraph/cmd/dgraphimport/import_client.go +++ b/dgraph/cmd/dgraphimport/import_client.go @@ -7,16 +7,13 @@ package dgraphimport import ( "context" - "errors" "fmt" - "io" - "math" "os" "path/filepath" "github.com/dgraph-io/badger/v4" apiv2 "github.com/dgraph-io/dgo/v250/protos/api.v2" - "github.com/dgraph-io/ristretto/v2/z" + "github.com/hypermodeinc/dgraph/v25/worker" "github.com/golang/glog" "golang.org/x/sync/errgroup" @@ -48,10 +45,12 @@ func Import(ctx context.Context, endpoint string, opts grpc.DialOption, bulkOutD } // startPDirStream initiates a snapshot stream session with the Dgraph server. -func startPDirStream(ctx context.Context, dc apiv2.DgraphClient) (*apiv2.InitiatePDirStreamResponse, error) { +func startPDirStream(ctx context.Context, dc apiv2.DgraphClient) (*apiv2.UpdateExtSnapshotStreamingStateResponse, error) { glog.Info("Initiating pdir stream") - req := &apiv2.InitiatePDirStreamRequest{} - resp, err := dc.InitiatePDirStream(ctx, req) + req := &apiv2.UpdateExtSnapshotStreamingStateRequest{ + Start: true, + } + resp, err := dc.UpdateExtSnapshotStreamingState(ctx, req) if err != nil { glog.Errorf("failed to initiate pdir stream: %v", err) return nil, fmt.Errorf("failed to initiate pdir stream: %v", err) @@ -63,7 +62,7 @@ func startPDirStream(ctx context.Context, dc apiv2.DgraphClient) (*apiv2.Initiat // sendPDir takes a p directory and a set of group IDs and streams the data from the // p directory to the corresponding group IDs. It first scans the provided directory for // subdirectories named with numeric group IDs. -func sendPDir(ctx context.Context, dg apiv2.DgraphClient, baseDir string, groups []uint32) error { +func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups []uint32) error { glog.Infof("Starting to stream pdir from directory: %s", baseDir) errG, ctx := errgroup.WithContext(ctx) @@ -74,9 +73,8 @@ func sendPDir(ctx context.Context, dg apiv2.DgraphClient, baseDir string, groups if _, err := os.Stat(pDir); err != nil { return fmt.Errorf("p directory does not exist for group [%d]: [%s]", group, pDir) } - glog.Infof("Streaming data for group [%d] from directory: [%s]", group, pDir) - if err := streamData(ctx, dg, pDir, group); err != nil { + if err := streamData(ctx, dc, pDir, group); err != nil { glog.Errorf("Failed to stream data for groups [%v] from directory: [%s]: %v", group, pDir, err) return err } @@ -84,21 +82,34 @@ func sendPDir(ctx context.Context, dg apiv2.DgraphClient, baseDir string, groups return nil }) } - if err := errG.Wait(); err != nil { - return err + if err1 := errG.Wait(); err1 != nil { + // If the p directory doesn't exist for this group, it indicates that + // streaming might be in progress to other groups. We disable drain mode + // to prevent interference and drop any streamed data to ensure a clean state. + req := &apiv2.UpdateExtSnapshotStreamingStateRequest{ + Start: false, + Finish: true, + DropData: true, + } + if _, err := dc.UpdateExtSnapshotStreamingState(context.Background(), req); err != nil { + return fmt.Errorf("failed to stream data :%v failed to off drain mode: %v", err1, err) + } + + glog.Info("successfully disabled drain mode") + return err1 } - glog.Infof("Completed streaming all pdirs") + glog.Info("Completed streaming all pdirs") return nil } // streamData handles the actual data streaming process for a single group. // It opens the BadgerDB at the specified directory and streams all data to the server. -func streamData(ctx context.Context, dg apiv2.DgraphClient, pdir string, groupId uint32) error { +func streamData(ctx context.Context, dc apiv2.DgraphClient, pdir string, groupId uint32) error { glog.Infof("Opening stream for group %d from directory %s", groupId, pdir) // Initialize stream with the server - out, err := dg.StreamPDir(ctx) + out, err := dc.StreamExtSnapshot(ctx) if err != nil { return fmt.Errorf("failed to start pdir stream for group %d: %w", groupId, err) } @@ -118,41 +129,23 @@ func streamData(ctx context.Context, dg apiv2.DgraphClient, pdir string, groupId // Send group ID as the first message in the stream glog.Infof("Sending group ID [%d] to server", groupId) - groupReq := &apiv2.StreamPDirRequest{GroupId: groupId} + groupReq := &apiv2.StreamExtSnapshotRequest{GroupId: groupId} if err := out.Send(groupReq); err != nil { return fmt.Errorf("failed to send group ID [%d]: %w", groupId, err) } // Configure and start the BadgerDB stream glog.Infof("Starting BadgerDB stream for group [%d]", groupId) - stream := ps.NewStreamAt(math.MaxUint64) - stream.LogPrefix = fmt.Sprintf("Sending P dir for group [%d]", groupId) - stream.KeyToList = nil - stream.Send = func(buf *z.Buffer) error { - p := &apiv2.StreamPacket{Data: buf.Bytes()} - if err := out.Send(&apiv2.StreamPDirRequest{StreamPacket: p}); err != nil && !errors.Is(err, io.EOF) { - return fmt.Errorf("failed to send data chunk: %w", err) - } - return nil - } - - // Execute the stream process - if err := stream.Orchestrate(ctx); err != nil { - return fmt.Errorf("stream orchestration failed for group [%d]: %w", groupId, err) - } - - // Send the final 'done' signal to mark completion - glog.Infof("Sending completion signal for group [%d]", groupId) - done := &apiv2.StreamPacket{Done: true} + // if err := RunBadgerStream(ctx, ps, out, groupId); err != nil { + // return fmt.Errorf("badger stream failed for group [%d]: %w", groupId, err) + // } - if err := out.Send(&apiv2.StreamPDirRequest{StreamPacket: done}); err != nil && !errors.Is(err, io.EOF) { - return fmt.Errorf("failed to send 'done' signal for group [%d]: %w", groupId, err) + if err := worker.RunBadgerStream(ctx, ps, out, groupId); err != nil { + return fmt.Errorf("badger stream failed for group [%d]: %w", groupId, err) } - // Wait for acknowledgment from the server if _, err := out.CloseAndRecv(); err != nil { return fmt.Errorf("failed to receive ACK for group [%d]: %w", groupId, err) } glog.Infof("Group [%d]: Received ACK ", groupId) - return nil } diff --git a/dgraph/cmd/dgraphimport/import_test.go b/dgraph/cmd/dgraphimport/import_test.go index ef0c561e2c1..d86443dc17c 100644 --- a/dgraph/cmd/dgraphimport/import_test.go +++ b/dgraph/cmd/dgraphimport/import_test.go @@ -9,16 +9,22 @@ package dgraphimport import ( "context" "encoding/json" + "fmt" "path/filepath" + "strconv" + "strings" "testing" + "time" "github.com/hypermodeinc/dgraph/v25/dgraphapi" "github.com/hypermodeinc/dgraph/v25/dgraphtest" + "github.com/hypermodeinc/dgraph/v25/protos/pb" "github.com/hypermodeinc/dgraph/v25/systest/1million/common" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + "google.golang.org/protobuf/encoding/protojson" ) const expectedSchema = `{ @@ -91,24 +97,136 @@ func TestDrainModeAfterStartSnapshotStream(t *testing.T) { // TestImportApis tests import functionality with different cluster configurations func TestImportApis(t *testing.T) { tests := []struct { - name string - bulkAlphas int - targetAlphas int - replicasFactor int + name string + bulkAlphas int // Number of alphas in source cluster + targetAlphas int // Number of alphas in target cluster + replicasFactor int // Number of replicas for each group + downAlphas int // Number of alphas to be shutdown + negativeTestCase bool // True if this is an expected failure case + description string + err string + waitForSnapshot bool }{ - {"SingleGroupSingleAlpha", 1, 1, 1}, - {"TwoGroupsSingleAlpha", 2, 2, 1}, - {"ThreeGroupsSingleAlpha", 3, 3, 1}, + { + name: "SingleGroupShutTwoAlphasPerGroup", + bulkAlphas: 1, + targetAlphas: 3, + replicasFactor: 3, + downAlphas: 2, + negativeTestCase: true, + description: "Single group with 3 alphas, shutdown 2 alphas", + err: "failed to initiate pdir stream", + }, + { + name: "TwoGroupShutTwoAlphasPerGroup", + bulkAlphas: 2, + targetAlphas: 6, + replicasFactor: 3, + downAlphas: 2, + negativeTestCase: true, + description: "Two groups with 3 alphas each, shutdown 2 alphas per group", + err: "failed to initiate pdir stream", + }, + { + name: "TwoGroupShutTwoAlphasPerGroupNoPDir", + bulkAlphas: 1, + targetAlphas: 6, + replicasFactor: 3, + downAlphas: 0, + negativeTestCase: true, + description: "Two groups with 3 alphas each, 1 p directory is not available", + err: "p directory does not exist for group [2]", + }, + { + name: "ThreeGroupShutTwoAlphasPerGroup", + bulkAlphas: 3, + targetAlphas: 9, + replicasFactor: 3, + downAlphas: 2, + negativeTestCase: true, + description: "Three groups with 3 alphas each, shutdown 2 alphas per group", + err: "failed to initiate pdir stream", + }, + { + name: "SingleGroupShutOneAlpha", + bulkAlphas: 1, + targetAlphas: 3, + replicasFactor: 3, + downAlphas: 1, + negativeTestCase: false, + description: "Single group with multiple alphas, shutdown 1 alpha", + err: "", + waitForSnapshot: true, + }, + { + name: "TwoGroupShutOneAlphaPerGroup", + bulkAlphas: 2, + targetAlphas: 6, + replicasFactor: 3, + downAlphas: 1, + negativeTestCase: false, + description: "Multiple groups with multiple alphas, shutdown 1 alphas per group", + err: "", + waitForSnapshot: true, + }, + { + name: "ThreeGroupShutOneAlphaPerGroup", + bulkAlphas: 3, + targetAlphas: 9, + replicasFactor: 3, + downAlphas: 1, + negativeTestCase: false, + description: "Three groups with 3 alphas each, shutdown 1 alpha per group", + err: "", + waitForSnapshot: true, + }, + { + name: "SingleGroupAllAlphasOnline", + bulkAlphas: 1, + targetAlphas: 3, + replicasFactor: 3, + downAlphas: 0, + negativeTestCase: false, + description: "Single group with multiple alphas, all alphas are online", + err: "", + }, + { + name: "TwoGroupAllAlphasOnline", + bulkAlphas: 2, + targetAlphas: 6, + replicasFactor: 3, + downAlphas: 0, + negativeTestCase: false, + description: "Multiple groups with multiple alphas, all alphas are online", + err: "", + }, + { + name: "ThreeGroupAllAlphasOnline", + bulkAlphas: 3, + targetAlphas: 9, + replicasFactor: 3, + downAlphas: 0, + negativeTestCase: false, + description: "Three groups with 3 alphas each, all alphas are online", + err: "", + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - runImportTest(t, tt.bulkAlphas, tt.targetAlphas, tt.replicasFactor) + if tt.negativeTestCase { + t.Logf("Running negative test case: %s", tt.description) + } else { + t.Logf("Running test case: %s", tt.description) + } + runImportTest(t, tt.bulkAlphas, tt.targetAlphas, tt.replicasFactor, tt.downAlphas, tt.negativeTestCase, + tt.err, tt.waitForSnapshot) }) } } -func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor int) { +func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDownAlphas int, negative bool, + errStr string, waitForSnapshot bool) { bulkCluster, baseDir := setupBulkCluster(t, bulkAlphas) defer func() { bulkCluster.Cleanup(t.Failed()) }() @@ -116,14 +234,96 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor int) { defer func() { targetCluster.Cleanup(t.Failed()) }() defer gcCleanup() + _, err := gc.Query("schema{}") + require.NoError(t, err) + url, err := targetCluster.GetAlphaGrpcEndpoint(0) require.NoError(t, err) outDir := filepath.Join(baseDir, "out") + // Get group information for all alphas + _, cleanup, err := targetCluster.Client() + require.NoError(t, err) + defer cleanup() + + // Get health status for all instances + hc, err := targetCluster.HTTPClient() + require.NoError(t, err) + var state pb.MembershipState + + healthResp, err := hc.GetAlphaState() + require.NoError(t, err) + require.NoError(t, protojson.Unmarshal(healthResp, &state)) + fmt.Println("Health response: ", string(healthResp)) + + // Group alphas by their group number + alphaGroups := make(map[uint32][]int) + for _, group := range state.Groups { + for _, member := range group.Members { + if strings.Contains(member.Addr, "alpha0") { + continue + } + alphaNum := strings.TrimPrefix(member.Addr, "alpha") + alphaNum = strings.TrimSuffix(alphaNum, ":7080") + alphaID, err := strconv.Atoi(alphaNum) + require.NoError(t, err) + alphaGroups[member.GroupId] = append(alphaGroups[member.GroupId], alphaID) + } + } + + // Shutdown specified number of alphas from each group + for group, alphas := range alphaGroups { + for i := 0; i < numDownAlphas; i++ { + alphaID := alphas[i] + t.Logf("Shutting down alpha %v from group %v", alphaID, group) + require.NoError(t, targetCluster.StopAlpha(alphaID)) + } + } + + if negative { + err := Import(context.Background(), url, grpc.WithTransportCredentials(insecure.NewCredentials()), outDir) + require.Error(t, err) + fmt.Println("Error: ", err) + require.ErrorContains(t, err, errStr) + return + } + require.NoError(t, Import(context.Background(), url, grpc.WithTransportCredentials(insecure.NewCredentials()), outDir)) - verifyImportResults(t, gc) + for group, alphas := range alphaGroups { + for i := 0; i < numDownAlphas; i++ { + alphaID := alphas[i] + t.Logf("Starting alpha %v from group %v", alphaID, group) + require.NoError(t, targetCluster.StartAlpha(alphaID)) + } + } + + require.NoError(t, targetCluster.HealthCheck(false)) + + if waitForSnapshot { + for grp, alphas := range alphaGroups { + for i := 0; i < numDownAlphas; i++ { + fmt.Println("Waiting for snapshot for alpha", alphas[i], "group", grp) + hc, err := targetCluster.GetAlphaHttpClient(alphas[i]) + require.NoError(t, err) + + prevTs, err := hc.GetCurrentSnapshotTs(uint64(grp)) + require.NoError(t, err) + // no need to check error because the cluster may have already taken a snapshot + _, _ = hc.WaitForSnapshot(uint64(grp), prevTs) + } + } + } + + t.Log("Import completed") + + for i := 0; i < targetAlphas; i++ { + gc, cleanup, err := targetCluster.AlphaClient(i) + require.NoError(t, err) + defer cleanup() + verifyImportResults(t, gc) + } } // setupBulkCluster creates and configures a cluster for bulk loading data @@ -156,7 +356,7 @@ func setupBulkCluster(t *testing.T, numAlphas int) (*dgraphtest.LocalCluster, st func setupTargetCluster(t *testing.T, numAlphas, replicasFactor int) (*dgraphtest.LocalCluster, *dgraphapi.GrpcClient, func()) { conf := dgraphtest.NewClusterConfig(). WithNumAlphas(numAlphas). - WithNumZeros(1). + WithNumZeros(3). WithReplicas(replicasFactor) cluster, err := dgraphtest.NewLocalCluster(conf) @@ -170,24 +370,49 @@ func setupTargetCluster(t *testing.T, numAlphas, replicasFactor int) (*dgraphtes return cluster, gc, cleanup } -// verifyImportResults validates the result of an import operation +// verifyImportResults validates the result of an import operation with retry logic func verifyImportResults(t *testing.T, gc *dgraphapi.GrpcClient) { - // Check schema after streaming process - schemaResp, err := gc.Query("schema{}") - require.NoError(t, err) - // Compare the schema response with the expected schema - var actualSchema, expectedSchemaObj map[string]interface{} - require.NoError(t, json.Unmarshal(schemaResp.Json, &actualSchema)) - require.NoError(t, json.Unmarshal([]byte(expectedSchema), &expectedSchemaObj)) + maxRetries := 10 + retryDelay := 500 * time.Millisecond + hasAllPredicates := true - // Check if the actual schema contains all the predicates from expected schema - actualPredicates := getPredicateMap(actualSchema) + // Get expected predicates first + var expectedSchemaObj map[string]interface{} + require.NoError(t, json.Unmarshal([]byte(expectedSchema), &expectedSchemaObj)) expectedPredicates := getPredicateMap(expectedSchemaObj) - for predName, predDetails := range expectedPredicates { - actualPred, exists := actualPredicates[predName] - require.True(t, exists, "Predicate '%s' not found in actual schema", predName) - require.Equal(t, predDetails, actualPred, "Predicate '%s' details don't match", predName) + for i := 0; i < maxRetries; i++ { + schemaResp, err := gc.Query("schema{}") + require.NoError(t, err) + + // Parse schema response + var actualSchema map[string]interface{} + require.NoError(t, json.Unmarshal(schemaResp.Json, &actualSchema)) + + // Get actual predicates + actualPredicates := getPredicateMap(actualSchema) + + // Check if all expected predicates are present + for predName := range expectedPredicates { + if _, exists := actualPredicates[predName]; !exists { + hasAllPredicates = false + break + } + } + + if hasAllPredicates { + break + } + + if i < maxRetries-1 { + t.Logf("Not all predicates found yet, retrying in %v", retryDelay) + time.Sleep(retryDelay) + retryDelay *= 2 + } + } + + if !hasAllPredicates { + t.Fatalf("Not all predicates found in schema") } for _, tt := range common.OneMillionTCs { diff --git a/dgraphapi/cluster.go b/dgraphapi/cluster.go index 3ec6a088529..2a7107b5fd0 100644 --- a/dgraphapi/cluster.go +++ b/dgraphapi/cluster.go @@ -57,11 +57,12 @@ type HttpToken struct { // HTTPClient allows doing operations on Dgraph over http type HTTPClient struct { *HttpToken - adminURL string - graphqlURL string - stateURL string - dqlURL string - dqlMutateUrl string + adminURL string + graphqlURL string + stateURL string + dqlURL string + dqlMutateUrl string + alphaStateUrl string } // GraphQLParams are used for making graphql requests to dgraph @@ -647,6 +648,25 @@ func (hc *HTTPClient) GetZeroState() (*LicenseResponse, error) { return &stateResponse, nil } +func (hc *HTTPClient) GetAlphaState() ([]byte, error) { + response, err := http.Get(hc.alphaStateUrl) + if err != nil { + return nil, errors.Wrap(err, "error getting alpha state http response") + } + defer func() { + if err := response.Body.Close(); err != nil { + log.Printf("[WARNING] error closing body: %v", err) + } + }() + + body, err := io.ReadAll(response.Body) + if err != nil { + return nil, errors.Wrapf(err, "error reading zero state response body") + } + + return body, nil +} + func (hc *HTTPClient) PostDqlQuery(query string) ([]byte, error) { req, err := http.NewRequest(http.MethodPost, hc.dqlURL, bytes.NewBufferString(query)) if err != nil { @@ -760,14 +780,17 @@ func GetHttpClient(alphaUrl, zeroUrl string) (*HTTPClient, error) { adminUrl := "http://" + alphaUrl + "/admin" graphQLUrl := "http://" + alphaUrl + "/graphql" stateUrl := "http://" + zeroUrl + "/state" + alphaStateUrl := "http://" + alphaUrl + "/state" + dqlUrl := "http://" + alphaUrl + "/query" dqlMutateUrl := "http://" + alphaUrl + "/mutate" return &HTTPClient{ - adminURL: adminUrl, - graphqlURL: graphQLUrl, - stateURL: stateUrl, - dqlURL: dqlUrl, - dqlMutateUrl: dqlMutateUrl, - HttpToken: &HttpToken{}, + adminURL: adminUrl, + graphqlURL: graphQLUrl, + stateURL: stateUrl, + dqlURL: dqlUrl, + dqlMutateUrl: dqlMutateUrl, + alphaStateUrl: alphaStateUrl, + HttpToken: &HttpToken{}, }, nil } diff --git a/dgraphtest/local_cluster.go b/dgraphtest/local_cluster.go index 73657dec6ed..93715588a00 100644 --- a/dgraphtest/local_cluster.go +++ b/dgraphtest/local_cluster.go @@ -837,6 +837,15 @@ func (c *LocalCluster) HTTPClient() (*dgraphapi.HTTPClient, error) { return dgraphapi.GetHttpClient(alphaUrl, zeroUrl) } +func (c *LocalCluster) GetAlphaHttpClient(alphaID int) (*dgraphapi.HTTPClient, error) { + pubPort, err := publicPort(c.dcli, c.alphas[alphaID], alphaHttpPort) + if err != nil { + return nil, err + } + url := "0.0.0.0:" + pubPort + return dgraphapi.GetHttpClient(url, "") +} + // serverURL returns url to the 'server' 'endpoint' func (c *LocalCluster) serverURL(server, endpoint string) (string, error) { pubPort, err := publicPort(c.dcli, c.alphas[0], alphaHttpPort) diff --git a/edgraph/server.go b/edgraph/server.go index c64619d8c25..42b125eccf7 100644 --- a/edgraph/server.go +++ b/edgraph/server.go @@ -1810,31 +1810,35 @@ func validateNamespace(ctx context.Context, tc *api.TxnContext) error { return nil } -func (s *ServerV25) InitiatePDirStream(ctx context.Context, - c *apiv2.InitiatePDirStreamRequest) (v *apiv2.InitiatePDirStreamResponse, err error) { +func (s *ServerV25) UpdateExtSnapshotStreamingState(ctx context.Context, + req *apiv2.UpdateExtSnapshotStreamingStateRequest) (v *apiv2.UpdateExtSnapshotStreamingStateResponse, err error) { + if req == nil { + return nil, errors.New("UpdateExtSnapshotStreamingStateRequest should not be empty") + } + + if req.Start && req.Finish { + return nil, errors.New("UpdateExtSnapshotStreamingStateRequest should not be both start and finish") + } - drainMode := &pb.DrainModeRequest{State: true} - groups, err := worker.ProposeDrain(ctx, drainMode) + drainModeReq := &apiv2.UpdateExtSnapshotStreamingStateRequest{ + Start: req.Start, + Finish: req.Finish, + DropData: req.DropData, + } + groups, err := worker.ProposeDrain(ctx, drainModeReq) if err != nil { return nil, err } - resp := &apiv2.InitiatePDirStreamResponse{Groups: groups} + resp := &apiv2.UpdateExtSnapshotStreamingStateResponse{Groups: groups} return resp, nil } -func (s *ServerV25) StreamPDir(stream apiv2.Dgraph_StreamPDirServer) error { - if err := worker.InStream(stream); err != nil { - return err - } - - drainMode := &pb.DrainModeRequest{State: false} - if _, err := worker.ProposeDrain(stream.Context(), drainMode); err != nil { - return err - } +func (s *ServerV25) StreamExtSnapshot(stream apiv2.Dgraph_StreamExtSnapshotServer) error { + defer x.ExtSnapshotStreamingState(false) - return nil + return worker.InStream(stream) } // CommitOrAbort commits or aborts a transaction. diff --git a/go.mod b/go.mod index fb92cd81272..a3eb4f08312 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/Masterminds/semver/v3 v3.3.1 github.com/blevesearch/bleve/v2 v2.5.1 github.com/dgraph-io/badger/v4 v4.7.0 - github.com/dgraph-io/dgo/v250 v250.0.0-preview4 + github.com/dgraph-io/dgo/v250 v250.0.0-preview4.0.20250529053946-12559fbb311c github.com/dgraph-io/gqlgen v0.13.2 github.com/dgraph-io/gqlparser/v2 v2.2.2 github.com/dgraph-io/graphql-transport-ws v0.0.0-20210511143556-2cef522f1f15 diff --git a/go.sum b/go.sum index 2f4362e7eac..4c91fd786fa 100644 --- a/go.sum +++ b/go.sum @@ -124,8 +124,8 @@ github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1 github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dgraph-io/badger/v4 v4.7.0 h1:Q+J8HApYAY7UMpL8d9owqiB+odzEc0zn/aqOD9jhc6Y= github.com/dgraph-io/badger/v4 v4.7.0/go.mod h1:He7TzG3YBy3j4f5baj5B7Zl2XyfNe5bl4Udl0aPemVA= -github.com/dgraph-io/dgo/v250 v250.0.0-preview4 h1:DkS6iFI6RwStXRzQxT5v8b6NLqqHQi0xKSK6FvcEwYo= -github.com/dgraph-io/dgo/v250 v250.0.0-preview4/go.mod h1:6nnKW4tYiai9xI6NSCrxaBgUGG1YI/+KlY+Tc7smqXY= +github.com/dgraph-io/dgo/v250 v250.0.0-preview4.0.20250529053946-12559fbb311c h1:s97uvlF0KFZ+X3vKhPLA+PqkTLGbaiNkuVB6/94MktY= +github.com/dgraph-io/dgo/v250 v250.0.0-preview4.0.20250529053946-12559fbb311c/go.mod h1:6nnKW4tYiai9xI6NSCrxaBgUGG1YI/+KlY+Tc7smqXY= github.com/dgraph-io/gqlgen v0.13.2 h1:TNhndk+eHKj5qE7BenKKSYdSIdOGhLqxR1rCiMso9KM= github.com/dgraph-io/gqlgen v0.13.2/go.mod h1:iCOrOv9lngN7KAo+jMgvUPVDlYHdf7qDwsTkQby2Sis= github.com/dgraph-io/gqlparser/v2 v2.1.1/go.mod h1:MYS4jppjyx8b9tuUtjV7jU1UFZK6P9fvO8TsIsQtRKU= diff --git a/protos/pb.proto b/protos/pb.proto index fd07824c6d5..acafd76877d 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -319,7 +319,7 @@ message Proposal { DeleteNsRequest delete_ns = 14; // Used to delete namespace. // Skipping 15 as it is used for uint64 key in master and might be needed later here. uint64 start_ts = 16; - DrainModeRequest drainmode = 17; + api.v2.UpdateExtSnapshotStreamingStateRequest update_ext_snapshot_streaming_state = 17; } message CDCState { @@ -596,12 +596,8 @@ service Worker { returns (UpdateGraphQLSchemaResponse) {} rpc DeleteNamespace(DeleteNsRequest) returns (Status) {} rpc TaskStatus(TaskStatusRequest) returns (TaskStatusResponse) {} - rpc ApplyDrainmode(DrainModeRequest) returns (Status) {} - rpc InternalStreamPDir(stream api.v2.StreamPDirRequest) returns (api.v2.StreamPDirResponse) {} -} - -message DrainModeRequest { - bool state = 1; + rpc UpdateExtSnapshotStreamingState(api.v2.UpdateExtSnapshotStreamingStateRequest) returns (Status) {} + rpc InternalStreamPDir(stream api.v2.StreamExtSnapshotRequest) returns (api.v2.StreamExtSnapshotResponse) {} } message TabletResponse { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 3eec694628d..0afaf6de848 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -420,7 +420,7 @@ func (x NumLeaseType) Number() protoreflect.EnumNumber { // Deprecated: Use NumLeaseType.Descriptor instead. func (NumLeaseType) EnumDescriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{56, 0} + return file_pb_proto_rawDescGZIP(), []int{55, 0} } type DropOperation_DropOp int32 @@ -472,7 +472,7 @@ func (x DropOperation_DropOp) Number() protoreflect.EnumNumber { // Deprecated: Use DropOperation_DropOp.Descriptor instead. func (DropOperation_DropOp) EnumDescriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{64, 0} + return file_pb_proto_rawDescGZIP(), []int{63, 0} } type BackupKey_KeyType int32 @@ -536,7 +536,7 @@ func (x BackupKey_KeyType) Number() protoreflect.EnumNumber { // Deprecated: Use BackupKey_KeyType.Descriptor instead. func (BackupKey_KeyType) EnumDescriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{67, 0} + return file_pb_proto_rawDescGZIP(), []int{66, 0} } type List struct { @@ -2692,8 +2692,8 @@ type Proposal struct { CdcState *CDCState `protobuf:"bytes,13,opt,name=cdc_state,json=cdcState,proto3" json:"cdc_state,omitempty"` DeleteNs *DeleteNsRequest `protobuf:"bytes,14,opt,name=delete_ns,json=deleteNs,proto3" json:"delete_ns,omitempty"` // Used to delete namespace. // Skipping 15 as it is used for uint64 key in master and might be needed later here. - StartTs uint64 `protobuf:"varint,16,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` - Drainmode *DrainModeRequest `protobuf:"bytes,17,opt,name=drainmode,proto3" json:"drainmode,omitempty"` + StartTs uint64 `protobuf:"varint,16,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` + UpdateExtSnapshotStreamingState *api_v2.UpdateExtSnapshotStreamingStateRequest `protobuf:"bytes,17,opt,name=update_ext_snapshot_streaming_state,json=updateExtSnapshotStreamingState,proto3" json:"update_ext_snapshot_streaming_state,omitempty"` } func (x *Proposal) Reset() { @@ -2812,9 +2812,9 @@ func (x *Proposal) GetStartTs() uint64 { return 0 } -func (x *Proposal) GetDrainmode() *DrainModeRequest { +func (x *Proposal) GetUpdateExtSnapshotStreamingState() *api_v2.UpdateExtSnapshotStreamingStateRequest { if x != nil { - return x.Drainmode + return x.UpdateExtSnapshotStreamingState } return nil } @@ -4565,53 +4565,6 @@ func (x *RaftBatch) GetPayload() *api.Payload { return nil } -type DrainModeRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - State bool `protobuf:"varint,1,opt,name=state,proto3" json:"state,omitempty"` -} - -func (x *DrainModeRequest) Reset() { - *x = DrainModeRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[51] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *DrainModeRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*DrainModeRequest) ProtoMessage() {} - -func (x *DrainModeRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[51] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use DrainModeRequest.ProtoReflect.Descriptor instead. -func (*DrainModeRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{51} -} - -func (x *DrainModeRequest) GetState() bool { - if x != nil { - return x.State - } - return false -} - type TabletResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4623,7 +4576,7 @@ type TabletResponse struct { func (x *TabletResponse) Reset() { *x = TabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[52] + mi := &file_pb_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4636,7 +4589,7 @@ func (x *TabletResponse) String() string { func (*TabletResponse) ProtoMessage() {} func (x *TabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[52] + mi := &file_pb_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4649,7 +4602,7 @@ func (x *TabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use TabletResponse.ProtoReflect.Descriptor instead. func (*TabletResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{52} + return file_pb_proto_rawDescGZIP(), []int{51} } func (x *TabletResponse) GetTablets() []*Tablet { @@ -4671,7 +4624,7 @@ type TabletRequest struct { func (x *TabletRequest) Reset() { *x = TabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[53] + mi := &file_pb_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4684,7 +4637,7 @@ func (x *TabletRequest) String() string { func (*TabletRequest) ProtoMessage() {} func (x *TabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[53] + mi := &file_pb_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4697,7 +4650,7 @@ func (x *TabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TabletRequest.ProtoReflect.Descriptor instead. func (*TabletRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{53} + return file_pb_proto_rawDescGZIP(), []int{52} } func (x *TabletRequest) GetTablets() []*Tablet { @@ -4726,7 +4679,7 @@ type SubscriptionRequest struct { func (x *SubscriptionRequest) Reset() { *x = SubscriptionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[54] + mi := &file_pb_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4739,7 +4692,7 @@ func (x *SubscriptionRequest) String() string { func (*SubscriptionRequest) ProtoMessage() {} func (x *SubscriptionRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[54] + mi := &file_pb_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4752,7 +4705,7 @@ func (x *SubscriptionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscriptionRequest.ProtoReflect.Descriptor instead. func (*SubscriptionRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{54} + return file_pb_proto_rawDescGZIP(), []int{53} } func (x *SubscriptionRequest) GetPrefixes() [][]byte { @@ -4780,7 +4733,7 @@ type SubscriptionResponse struct { func (x *SubscriptionResponse) Reset() { *x = SubscriptionResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[55] + mi := &file_pb_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4793,7 +4746,7 @@ func (x *SubscriptionResponse) String() string { func (*SubscriptionResponse) ProtoMessage() {} func (x *SubscriptionResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[55] + mi := &file_pb_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4806,7 +4759,7 @@ func (x *SubscriptionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SubscriptionResponse.ProtoReflect.Descriptor instead. func (*SubscriptionResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{55} + return file_pb_proto_rawDescGZIP(), []int{54} } func (x *SubscriptionResponse) GetKvs() *pb.KVList { @@ -4833,7 +4786,7 @@ type Num struct { func (x *Num) Reset() { *x = Num{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[56] + mi := &file_pb_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4846,7 +4799,7 @@ func (x *Num) String() string { func (*Num) ProtoMessage() {} func (x *Num) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[56] + mi := &file_pb_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4859,7 +4812,7 @@ func (x *Num) ProtoReflect() protoreflect.Message { // Deprecated: Use Num.ProtoReflect.Descriptor instead. func (*Num) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{56} + return file_pb_proto_rawDescGZIP(), []int{55} } func (x *Num) GetVal() uint64 { @@ -4911,7 +4864,7 @@ type AssignedIds struct { func (x *AssignedIds) Reset() { *x = AssignedIds{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[57] + mi := &file_pb_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4924,7 +4877,7 @@ func (x *AssignedIds) String() string { func (*AssignedIds) ProtoMessage() {} func (x *AssignedIds) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[57] + mi := &file_pb_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4937,7 +4890,7 @@ func (x *AssignedIds) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignedIds.ProtoReflect.Descriptor instead. func (*AssignedIds) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{57} + return file_pb_proto_rawDescGZIP(), []int{56} } func (x *AssignedIds) GetStartId() uint64 { @@ -4973,7 +4926,7 @@ type RemoveNodeRequest struct { func (x *RemoveNodeRequest) Reset() { *x = RemoveNodeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[58] + mi := &file_pb_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4986,7 +4939,7 @@ func (x *RemoveNodeRequest) String() string { func (*RemoveNodeRequest) ProtoMessage() {} func (x *RemoveNodeRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[58] + mi := &file_pb_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4999,7 +4952,7 @@ func (x *RemoveNodeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveNodeRequest.ProtoReflect.Descriptor instead. func (*RemoveNodeRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{58} + return file_pb_proto_rawDescGZIP(), []int{57} } func (x *RemoveNodeRequest) GetNodeId() uint64 { @@ -5029,7 +4982,7 @@ type MoveTabletRequest struct { func (x *MoveTabletRequest) Reset() { *x = MoveTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[59] + mi := &file_pb_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5042,7 +4995,7 @@ func (x *MoveTabletRequest) String() string { func (*MoveTabletRequest) ProtoMessage() {} func (x *MoveTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[59] + mi := &file_pb_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5055,7 +5008,7 @@ func (x *MoveTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MoveTabletRequest.ProtoReflect.Descriptor instead. func (*MoveTabletRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{59} + return file_pb_proto_rawDescGZIP(), []int{58} } func (x *MoveTabletRequest) GetNamespace() uint64 { @@ -5091,7 +5044,7 @@ type SnapshotMeta struct { func (x *SnapshotMeta) Reset() { *x = SnapshotMeta{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[60] + mi := &file_pb_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5104,7 +5057,7 @@ func (x *SnapshotMeta) String() string { func (*SnapshotMeta) ProtoMessage() {} func (x *SnapshotMeta) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[60] + mi := &file_pb_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5117,7 +5070,7 @@ func (x *SnapshotMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use SnapshotMeta.ProtoReflect.Descriptor instead. func (*SnapshotMeta) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{60} + return file_pb_proto_rawDescGZIP(), []int{59} } func (x *SnapshotMeta) GetClientTs() uint64 { @@ -5148,7 +5101,7 @@ type Status struct { func (x *Status) Reset() { *x = Status{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[61] + mi := &file_pb_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5161,7 +5114,7 @@ func (x *Status) String() string { func (*Status) ProtoMessage() {} func (x *Status) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[61] + mi := &file_pb_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5174,7 +5127,7 @@ func (x *Status) ProtoReflect() protoreflect.Message { // Deprecated: Use Status.ProtoReflect.Descriptor instead. func (*Status) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{61} + return file_pb_proto_rawDescGZIP(), []int{60} } func (x *Status) GetCode() int32 { @@ -5220,7 +5173,7 @@ type BackupRequest struct { func (x *BackupRequest) Reset() { *x = BackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[62] + mi := &file_pb_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5233,7 +5186,7 @@ func (x *BackupRequest) String() string { func (*BackupRequest) ProtoMessage() {} func (x *BackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[62] + mi := &file_pb_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5246,7 +5199,7 @@ func (x *BackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupRequest.ProtoReflect.Descriptor instead. func (*BackupRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{62} + return file_pb_proto_rawDescGZIP(), []int{61} } func (x *BackupRequest) GetReadTs() uint64 { @@ -5337,7 +5290,7 @@ type BackupResponse struct { func (x *BackupResponse) Reset() { *x = BackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[63] + mi := &file_pb_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5350,7 +5303,7 @@ func (x *BackupResponse) String() string { func (*BackupResponse) ProtoMessage() {} func (x *BackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[63] + mi := &file_pb_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5363,7 +5316,7 @@ func (x *BackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupResponse.ProtoReflect.Descriptor instead. func (*BackupResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{63} + return file_pb_proto_rawDescGZIP(), []int{62} } func (x *BackupResponse) GetDropOperations() []*DropOperation { @@ -5387,7 +5340,7 @@ type DropOperation struct { func (x *DropOperation) Reset() { *x = DropOperation{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[64] + mi := &file_pb_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5400,7 +5353,7 @@ func (x *DropOperation) String() string { func (*DropOperation) ProtoMessage() {} func (x *DropOperation) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[64] + mi := &file_pb_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5413,7 +5366,7 @@ func (x *DropOperation) ProtoReflect() protoreflect.Message { // Deprecated: Use DropOperation.ProtoReflect.Descriptor instead. func (*DropOperation) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{64} + return file_pb_proto_rawDescGZIP(), []int{63} } func (x *DropOperation) GetDropOp() DropOperation_DropOp { @@ -5451,7 +5404,7 @@ type ExportRequest struct { func (x *ExportRequest) Reset() { *x = ExportRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[65] + mi := &file_pb_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5464,7 +5417,7 @@ func (x *ExportRequest) String() string { func (*ExportRequest) ProtoMessage() {} func (x *ExportRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[65] + mi := &file_pb_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5477,7 +5430,7 @@ func (x *ExportRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExportRequest.ProtoReflect.Descriptor instead. func (*ExportRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{65} + return file_pb_proto_rawDescGZIP(), []int{64} } func (x *ExportRequest) GetGroupId() uint32 { @@ -5564,7 +5517,7 @@ type ExportResponse struct { func (x *ExportResponse) Reset() { *x = ExportResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[66] + mi := &file_pb_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5577,7 +5530,7 @@ func (x *ExportResponse) String() string { func (*ExportResponse) ProtoMessage() {} func (x *ExportResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[66] + mi := &file_pb_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5590,7 +5543,7 @@ func (x *ExportResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExportResponse.ProtoReflect.Descriptor instead. func (*ExportResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{66} + return file_pb_proto_rawDescGZIP(), []int{65} } func (x *ExportResponse) GetCode() int32 { @@ -5632,7 +5585,7 @@ type BackupKey struct { func (x *BackupKey) Reset() { *x = BackupKey{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[67] + mi := &file_pb_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5645,7 +5598,7 @@ func (x *BackupKey) String() string { func (*BackupKey) ProtoMessage() {} func (x *BackupKey) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[67] + mi := &file_pb_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5658,7 +5611,7 @@ func (x *BackupKey) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupKey.ProtoReflect.Descriptor instead. func (*BackupKey) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{67} + return file_pb_proto_rawDescGZIP(), []int{66} } func (x *BackupKey) GetType() BackupKey_KeyType { @@ -5726,7 +5679,7 @@ type BackupPostingList struct { func (x *BackupPostingList) Reset() { *x = BackupPostingList{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[68] + mi := &file_pb_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5739,7 +5692,7 @@ func (x *BackupPostingList) String() string { func (*BackupPostingList) ProtoMessage() {} func (x *BackupPostingList) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[68] + mi := &file_pb_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5752,7 +5705,7 @@ func (x *BackupPostingList) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupPostingList.ProtoReflect.Descriptor instead. func (*BackupPostingList) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{68} + return file_pb_proto_rawDescGZIP(), []int{67} } func (x *BackupPostingList) GetUids() []uint64 { @@ -5804,7 +5757,7 @@ type UpdateGraphQLSchemaRequest struct { func (x *UpdateGraphQLSchemaRequest) Reset() { *x = UpdateGraphQLSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[69] + mi := &file_pb_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5817,7 +5770,7 @@ func (x *UpdateGraphQLSchemaRequest) String() string { func (*UpdateGraphQLSchemaRequest) ProtoMessage() {} func (x *UpdateGraphQLSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[69] + mi := &file_pb_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5830,7 +5783,7 @@ func (x *UpdateGraphQLSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateGraphQLSchemaRequest.ProtoReflect.Descriptor instead. func (*UpdateGraphQLSchemaRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{69} + return file_pb_proto_rawDescGZIP(), []int{68} } func (x *UpdateGraphQLSchemaRequest) GetStartTs() uint64 { @@ -5872,7 +5825,7 @@ type UpdateGraphQLSchemaResponse struct { func (x *UpdateGraphQLSchemaResponse) Reset() { *x = UpdateGraphQLSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[70] + mi := &file_pb_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5885,7 +5838,7 @@ func (x *UpdateGraphQLSchemaResponse) String() string { func (*UpdateGraphQLSchemaResponse) ProtoMessage() {} func (x *UpdateGraphQLSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[70] + mi := &file_pb_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5898,7 +5851,7 @@ func (x *UpdateGraphQLSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateGraphQLSchemaResponse.ProtoReflect.Descriptor instead. func (*UpdateGraphQLSchemaResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{70} + return file_pb_proto_rawDescGZIP(), []int{69} } func (x *UpdateGraphQLSchemaResponse) GetUid() uint64 { @@ -5922,7 +5875,7 @@ type BulkMeta struct { func (x *BulkMeta) Reset() { *x = BulkMeta{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[71] + mi := &file_pb_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5935,7 +5888,7 @@ func (x *BulkMeta) String() string { func (*BulkMeta) ProtoMessage() {} func (x *BulkMeta) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[71] + mi := &file_pb_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5948,7 +5901,7 @@ func (x *BulkMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use BulkMeta.ProtoReflect.Descriptor instead. func (*BulkMeta) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{71} + return file_pb_proto_rawDescGZIP(), []int{70} } func (x *BulkMeta) GetEdgeCount() int64 { @@ -5984,7 +5937,7 @@ type DeleteNsRequest struct { func (x *DeleteNsRequest) Reset() { *x = DeleteNsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[72] + mi := &file_pb_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5997,7 +5950,7 @@ func (x *DeleteNsRequest) String() string { func (*DeleteNsRequest) ProtoMessage() {} func (x *DeleteNsRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[72] + mi := &file_pb_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6010,7 +5963,7 @@ func (x *DeleteNsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteNsRequest.ProtoReflect.Descriptor instead. func (*DeleteNsRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{72} + return file_pb_proto_rawDescGZIP(), []int{71} } func (x *DeleteNsRequest) GetGroupId() uint32 { @@ -6038,7 +5991,7 @@ type TaskStatusRequest struct { func (x *TaskStatusRequest) Reset() { *x = TaskStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[73] + mi := &file_pb_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6051,7 +6004,7 @@ func (x *TaskStatusRequest) String() string { func (*TaskStatusRequest) ProtoMessage() {} func (x *TaskStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[73] + mi := &file_pb_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6064,7 +6017,7 @@ func (x *TaskStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TaskStatusRequest.ProtoReflect.Descriptor instead. func (*TaskStatusRequest) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{73} + return file_pb_proto_rawDescGZIP(), []int{72} } func (x *TaskStatusRequest) GetTaskId() uint64 { @@ -6085,7 +6038,7 @@ type TaskStatusResponse struct { func (x *TaskStatusResponse) Reset() { *x = TaskStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pb_proto_msgTypes[74] + mi := &file_pb_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6098,7 +6051,7 @@ func (x *TaskStatusResponse) String() string { func (*TaskStatusResponse) ProtoMessage() {} func (x *TaskStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_pb_proto_msgTypes[74] + mi := &file_pb_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6111,7 +6064,7 @@ func (x *TaskStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use TaskStatusResponse.ProtoReflect.Descriptor instead. func (*TaskStatusResponse) Descriptor() ([]byte, []int) { - return file_pb_proto_rawDescGZIP(), []int{74} + return file_pb_proto_rawDescGZIP(), []int{73} } func (x *TaskStatusResponse) GetTaskMeta() uint64 { @@ -6498,7 +6451,7 @@ var file_pb_proto_rawDesc = []byte{ 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x77, 0x61, 0x72, 0x65, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x69, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x77, 0x61, 0x72, 0x65, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, - 0x65, 0x22, 0x9e, 0x04, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x61, 0x6c, 0x12, 0x2b, + 0x65, 0x22, 0xe8, 0x04, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x61, 0x6c, 0x12, 0x2b, 0x0a, 0x09, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x09, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x02, 0x6b, @@ -6528,228 +6481,230 @@ var file_pb_proto_rawDesc = []byte{ 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x12, 0x32, 0x0a, 0x09, 0x64, 0x72, 0x61, 0x69, - 0x6e, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x62, - 0x2e, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x52, 0x09, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x6d, 0x6f, 0x64, 0x65, 0x4a, 0x04, 0x08, 0x07, - 0x10, 0x08, 0x22, 0x23, 0x0a, 0x08, 0x43, 0x44, 0x43, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, - 0x0a, 0x07, 0x73, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x06, 0x73, 0x65, 0x6e, 0x74, 0x54, 0x73, 0x22, 0x63, 0x0a, 0x03, 0x4b, 0x56, 0x53, 0x12, 0x12, - 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, - 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, - 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, - 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x22, 0x80, 0x04, 0x0a, - 0x07, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x06, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x2e, 0x0a, 0x08, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, - 0x56, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x76, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x3a, 0x0a, 0x0c, 0x70, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x0b, 0x70, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, - 0x6c, 0x61, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, - 0x6c, 0x61, 0x6e, 0x67, 0x54, 0x61, 0x67, 0x12, 0x22, 0x0a, 0x06, 0x66, 0x61, 0x63, 0x65, 0x74, - 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x46, 0x61, - 0x63, 0x65, 0x74, 0x52, 0x06, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x6f, - 0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, - 0x5f, 0x74, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x69, - 0x74, 0x54, 0x73, 0x22, 0xa0, 0x01, 0x0a, 0x07, 0x56, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x0b, 0x0a, 0x07, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, - 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x49, 0x4e, 0x54, 0x10, - 0x02, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x03, 0x12, 0x08, 0x0a, 0x04, - 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x41, 0x54, 0x45, 0x54, 0x49, - 0x4d, 0x45, 0x10, 0x05, 0x12, 0x07, 0x0a, 0x03, 0x47, 0x45, 0x4f, 0x10, 0x06, 0x12, 0x07, 0x0a, - 0x03, 0x55, 0x49, 0x44, 0x10, 0x07, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x41, 0x53, 0x53, 0x57, 0x4f, - 0x52, 0x44, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x09, - 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x0a, 0x12, 0x0c, 0x0a, 0x08, - 0x42, 0x49, 0x47, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x0b, 0x12, 0x0a, 0x0a, 0x06, 0x56, 0x46, - 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x0c, 0x22, 0x31, 0x0a, 0x0b, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, - 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x45, 0x46, 0x10, 0x00, 0x12, 0x09, - 0x0a, 0x05, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x56, 0x41, 0x4c, - 0x55, 0x45, 0x5f, 0x4c, 0x41, 0x4e, 0x47, 0x10, 0x02, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x22, - 0x51, 0x0a, 0x08, 0x55, 0x69, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x62, - 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, - 0x16, 0x0a, 0x06, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x06, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x75, - 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x55, 0x69, - 0x64, 0x73, 0x22, 0x6b, 0x0a, 0x07, 0x55, 0x69, 0x64, 0x50, 0x61, 0x63, 0x6b, 0x12, 0x1d, 0x0a, - 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x24, 0x0a, 0x06, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, - 0x62, 0x2e, 0x55, 0x69, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x06, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x5f, 0x72, 0x65, 0x66, 0x18, - 0x17, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x52, 0x65, 0x66, 0x22, - 0x8c, 0x01, 0x0a, 0x0b, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, - 0x1f, 0x0a, 0x04, 0x70, 0x61, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, - 0x70, 0x62, 0x2e, 0x55, 0x69, 0x64, 0x50, 0x61, 0x63, 0x6b, 0x52, 0x04, 0x70, 0x61, 0x63, 0x6b, - 0x12, 0x27, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, - 0x08, 0x70, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x6f, - 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x04, 0x52, 0x06, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x22, 0x34, - 0x0a, 0x0a, 0x46, 0x61, 0x63, 0x65, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x61, - 0x6c, 0x69, 0x61, 0x73, 0x22, 0x4e, 0x0a, 0x0b, 0x46, 0x61, 0x63, 0x65, 0x74, 0x50, 0x61, 0x72, - 0x61, 0x6d, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x6c, 0x6c, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x61, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x24, - 0x0a, 0x05, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, - 0x70, 0x62, 0x2e, 0x46, 0x61, 0x63, 0x65, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x52, 0x05, 0x70, - 0x61, 0x72, 0x61, 0x6d, 0x22, 0x2c, 0x0a, 0x06, 0x46, 0x61, 0x63, 0x65, 0x74, 0x73, 0x12, 0x22, - 0x0a, 0x06, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x46, 0x61, 0x63, 0x65, 0x74, 0x52, 0x06, 0x66, 0x61, 0x63, 0x65, - 0x74, 0x73, 0x22, 0x39, 0x0a, 0x0a, 0x46, 0x61, 0x63, 0x65, 0x74, 0x73, 0x4c, 0x69, 0x73, 0x74, - 0x12, 0x2b, 0x0a, 0x0b, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x61, 0x63, 0x65, 0x74, - 0x73, 0x52, 0x0a, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x22, 0x44, 0x0a, - 0x08, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x12, 0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x61, - 0x72, 0x67, 0x73, 0x22, 0x6a, 0x0a, 0x0a, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x72, 0x65, - 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x6f, - 0x70, 0x12, 0x2a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, - 0x72, 0x65, 0x65, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x12, 0x20, 0x0a, - 0x04, 0x66, 0x75, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, - 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x22, - 0x78, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x19, 0x0a, 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x07, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x70, - 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, - 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x65, - 0x6c, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x22, 0xd1, 0x02, 0x0a, 0x0a, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x64, - 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x65, - 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x18, 0x04, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x12, 0x18, - 0x0a, 0x07, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x07, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, - 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x6c, 0x69, - 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x18, 0x08, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x06, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x61, - 0x6e, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x12, 0x1f, - 0x0a, 0x0b, 0x6e, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0a, 0x6e, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x12, - 0x16, 0x0a, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, - 0x62, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, - 0x63, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x3a, 0x0a, - 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x2a, 0x0a, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, - 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x02, 0x18, - 0x01, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc1, 0x04, 0x0a, 0x0c, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, - 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, - 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x32, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x70, + 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x12, 0x7c, 0x0a, 0x23, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x5f, 0x65, 0x78, 0x74, 0x5f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, + 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x1f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, + 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x23, 0x0a, 0x08, + 0x43, 0x44, 0x43, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x65, 0x6e, 0x74, + 0x5f, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x73, 0x65, 0x6e, 0x74, 0x54, + 0x73, 0x22, 0x63, 0x0a, 0x03, 0x4b, 0x56, 0x53, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, + 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, + 0x12, 0x14, 0x0a, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x22, 0x80, 0x04, 0x0a, 0x07, 0x50, 0x6f, 0x73, 0x74, 0x69, + 0x6e, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x06, 0x52, + 0x03, 0x75, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x76, 0x61, + 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x61, 0x6c, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x38, 0x0a, 0x09, - 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x52, 0x09, 0x64, 0x69, 0x72, - 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x69, - 0x7a, 0x65, 0x72, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, - 0x69, 0x7a, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x69, - 0x73, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x12, 0x16, - 0x0a, 0x06, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, - 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x18, 0x09, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x6e, - 0x69, 0x71, 0x75, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x6e, 0x69, 0x71, - 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, - 0x6c, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x6e, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, - 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x6e, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, - 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0f, 0x6e, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x69, 0x73, - 0x74, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, - 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0a, 0x6e, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x12, 0x34, 0x0a, 0x0b, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, - 0x63, 0x73, 0x22, 0x39, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x12, - 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x44, - 0x45, 0x58, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, 0x52, 0x53, 0x45, 0x10, - 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x03, 0x4a, 0x04, 0x08, - 0x07, 0x10, 0x08, 0x52, 0x08, 0x65, 0x78, 0x70, 0x6c, 0x69, 0x63, 0x69, 0x74, 0x22, 0x4f, 0x0a, - 0x0f, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x34, - 0x0a, 0x0a, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x22, 0x53, 0x0a, 0x0a, 0x54, 0x79, 0x70, 0x65, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x28, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x22, 0x32, 0x0a, 0x09, 0x4d, 0x61, 0x70, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x0d, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x22, 0xb2, 0x01, - 0x0a, 0x14, 0x4d, 0x6f, 0x76, 0x65, 0x50, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x50, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, - 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, - 0x63, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x47, 0x69, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x65, 0x73, 0x74, 0x5f, 0x67, 0x69, 0x64, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x64, 0x65, 0x73, 0x74, 0x47, 0x69, 0x64, 0x12, 0x15, - 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, - 0x74, 0x78, 0x6e, 0x54, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x10, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, - 0x75, 0x6d, 0x22, 0x43, 0x0a, 0x09, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, - 0x19, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, - 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, - 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x0b, 0x4f, 0x72, 0x61, 0x63, - 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x04, 0x74, 0x78, 0x6e, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x78, 0x6e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x52, 0x04, 0x74, 0x78, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x61, - 0x78, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0b, 0x6d, 0x61, 0x78, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x12, 0x4c, 0x0a, - 0x0f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x72, 0x61, 0x63, - 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x67, 0x72, 0x6f, - 0x75, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x1f, - 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, - 0x0e, 0x0a, 0x02, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x04, 0x52, 0x02, 0x74, 0x73, 0x22, - 0x26, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x5e, 0x0a, 0x09, 0x52, 0x61, 0x66, 0x74, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x12, 0x29, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x43, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, - 0x26, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x07, - 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x28, 0x0a, 0x10, 0x44, 0x72, 0x61, 0x69, 0x6e, - 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x22, 0x36, 0x0a, 0x0e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x65, 0x52, 0x07, 0x76, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x3a, 0x0a, 0x0c, 0x70, 0x6f, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x17, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x70, 0x6f, 0x73, 0x74, 0x69, + 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x61, 0x6e, 0x67, 0x5f, 0x74, + 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x6c, 0x61, 0x6e, 0x67, 0x54, 0x61, + 0x67, 0x12, 0x22, 0x0a, 0x06, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x0a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x46, 0x61, 0x63, 0x65, 0x74, 0x52, 0x06, 0x66, + 0x61, 0x63, 0x65, 0x74, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, + 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, + 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x0e, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x73, 0x22, 0xa0, 0x01, + 0x0a, 0x07, 0x56, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x45, 0x46, + 0x41, 0x55, 0x4c, 0x54, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, + 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x49, 0x4e, 0x54, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x46, + 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x03, 0x12, 0x08, 0x0a, 0x04, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x04, + 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x41, 0x54, 0x45, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x05, 0x12, 0x07, + 0x0a, 0x03, 0x47, 0x45, 0x4f, 0x10, 0x06, 0x12, 0x07, 0x0a, 0x03, 0x55, 0x49, 0x44, 0x10, 0x07, + 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x41, 0x53, 0x53, 0x57, 0x4f, 0x52, 0x44, 0x10, 0x08, 0x12, 0x0a, + 0x0a, 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x09, 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x42, + 0x4a, 0x45, 0x43, 0x54, 0x10, 0x0a, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, 0x47, 0x46, 0x4c, 0x4f, + 0x41, 0x54, 0x10, 0x0b, 0x12, 0x0a, 0x0a, 0x06, 0x56, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x0c, + 0x22, 0x31, 0x0a, 0x0b, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x07, 0x0a, 0x03, 0x52, 0x45, 0x46, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x41, 0x4c, 0x55, + 0x45, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x4c, 0x41, 0x4e, + 0x47, 0x10, 0x02, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x22, 0x51, 0x0a, 0x08, 0x55, 0x69, 0x64, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x65, 0x6c, + 0x74, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x64, 0x65, 0x6c, 0x74, 0x61, + 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x75, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x55, 0x69, 0x64, 0x73, 0x22, 0x6b, 0x0a, 0x07, + 0x55, 0x69, 0x64, 0x50, 0x61, 0x63, 0x6b, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x62, 0x6c, 0x6f, + 0x63, 0x6b, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x55, 0x69, 0x64, 0x42, + 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x06, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x12, 0x1b, 0x0a, 0x09, + 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x17, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x08, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x52, 0x65, 0x66, 0x22, 0x8c, 0x01, 0x0a, 0x0b, 0x50, 0x6f, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x04, 0x70, 0x61, 0x63, + 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x55, 0x69, 0x64, + 0x50, 0x61, 0x63, 0x6b, 0x52, 0x04, 0x70, 0x61, 0x63, 0x6b, 0x12, 0x27, 0x0a, 0x08, 0x70, 0x6f, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, + 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x74, 0x69, + 0x6e, 0x67, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x74, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x73, + 0x12, 0x16, 0x0a, 0x06, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x04, + 0x52, 0x06, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x22, 0x34, 0x0a, 0x0a, 0x46, 0x61, 0x63, 0x65, + 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x4e, + 0x0a, 0x0b, 0x46, 0x61, 0x63, 0x65, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x19, 0x0a, + 0x08, 0x61, 0x6c, 0x6c, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x07, 0x61, 0x6c, 0x6c, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x24, 0x0a, 0x05, 0x70, 0x61, 0x72, 0x61, + 0x6d, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x61, 0x63, + 0x65, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x52, 0x05, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x22, 0x2c, + 0x0a, 0x06, 0x46, 0x61, 0x63, 0x65, 0x74, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x66, 0x61, 0x63, 0x65, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x46, + 0x61, 0x63, 0x65, 0x74, 0x52, 0x06, 0x66, 0x61, 0x63, 0x65, 0x74, 0x73, 0x22, 0x39, 0x0a, 0x0a, + 0x46, 0x61, 0x63, 0x65, 0x74, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x0b, 0x66, 0x61, + 0x63, 0x65, 0x74, 0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x61, 0x63, 0x65, 0x74, 0x73, 0x52, 0x0a, 0x66, 0x61, 0x63, + 0x65, 0x74, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x22, 0x44, 0x0a, 0x08, 0x46, 0x75, 0x6e, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x72, 0x67, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x61, 0x72, 0x67, 0x73, 0x22, 0x6a, 0x0a, + 0x0a, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x72, 0x65, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x6f, + 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x6f, 0x70, 0x12, 0x2a, 0x0a, 0x08, 0x63, + 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, + 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x72, 0x65, 0x65, 0x52, 0x08, 0x63, + 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x12, 0x20, 0x0a, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x22, 0x78, 0x0a, 0x0d, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, + 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x14, 0x0a, + 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x74, 0x79, + 0x70, 0x65, 0x73, 0x22, 0xd1, 0x02, 0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4e, 0x6f, + 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x74, + 0x6f, 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x76, 0x65, + 0x72, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x76, 0x65, 0x72, + 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x70, + 0x73, 0x65, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x6f, 0x5f, 0x63, + 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x6e, + 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x6e, 0x69, + 0x71, 0x75, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, + 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, + 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x56, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0a, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x3a, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x02, 0x18, 0x01, 0x52, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x22, 0xc1, 0x04, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, + 0x74, 0x65, 0x12, 0x32, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x73, 0x74, + 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x38, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, + 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x44, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x69, 0x76, 0x65, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x69, 0x7a, 0x65, 0x72, 0x12, 0x14, + 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x70, 0x73, 0x65, + 0x72, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, + 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x61, 0x6e, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, + 0x6c, 0x61, 0x6e, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x18, 0x0e, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x6e, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x6e, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, + 0x2a, 0x0a, 0x11, 0x6e, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x6c, 0x69, 0x73, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x6e, 0x6f, 0x6e, 0x4e, + 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x6f, + 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, + 0x6c, 0x69, 0x63, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x6e, 0x6f, 0x43, 0x6f, + 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x12, 0x34, 0x0a, 0x0b, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, + 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, + 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x39, 0x0a, 0x09, + 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, + 0x45, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x01, 0x12, 0x0b, + 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, 0x52, 0x53, 0x45, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, + 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x03, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x52, 0x08, 0x65, + 0x78, 0x70, 0x6c, 0x69, 0x63, 0x69, 0x74, 0x22, 0x4f, 0x0a, 0x0f, 0x56, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x70, 0x65, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, + 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, 0x52, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x34, 0x0a, 0x0a, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x53, + 0x0a, 0x0a, 0x54, 0x79, 0x70, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1b, 0x0a, 0x09, + 0x74, 0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x74, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x06, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x06, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x73, 0x22, 0x32, 0x0a, 0x09, 0x4d, 0x61, 0x70, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, + 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x22, 0xb2, 0x01, 0x0a, 0x14, 0x4d, 0x6f, 0x76, 0x65, + 0x50, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x69, 0x64, 0x12, 0x19, 0x0a, + 0x08, 0x64, 0x65, 0x73, 0x74, 0x5f, 0x67, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x07, 0x64, 0x65, 0x73, 0x74, 0x47, 0x69, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, + 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x54, 0x73, 0x12, + 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x65, 0x78, 0x70, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x22, 0x43, 0x0a, 0x09, + 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, + 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x0b, 0x4f, 0x72, 0x61, 0x63, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x12, 0x21, 0x0a, 0x04, 0x74, 0x78, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x04, + 0x74, 0x78, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x61, 0x78, 0x5f, 0x61, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6d, 0x61, 0x78, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x12, 0x4c, 0x0a, 0x0f, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x72, 0x61, 0x63, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x1f, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x74, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x04, 0x52, 0x02, 0x74, 0x73, 0x22, 0x26, 0x0a, 0x0c, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x5e, 0x0a, 0x09, 0x52, 0x61, 0x66, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x29, + 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x26, 0x0a, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x22, 0x36, 0x0a, 0x0e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x07, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x07, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0x50, 0x0a, 0x0d, 0x54, 0x61, 0x62, @@ -6985,7 +6940,7 @@ var file_pb_proto_rawDesc = []byte{ 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x31, 0x0a, 0x0a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0a, 0x2e, 0x70, 0x62, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0xf5, 0x06, 0x0a, 0x06, 0x57, 0x6f, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0xae, 0x07, 0x0a, 0x06, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x2a, 0x0a, 0x06, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x22, 0x00, 0x12, @@ -7032,16 +6987,20 @@ var file_pb_proto_rawDesc = []byte{ 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x34, 0x0a, 0x0e, 0x41, 0x70, 0x70, - 0x6c, 0x79, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x2e, 0x70, 0x62, - 0x2e, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, - 0x4f, 0x0a, 0x12, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x50, 0x44, 0x69, 0x72, 0x12, 0x19, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x44, 0x69, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x50, 0x44, 0x69, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, - 0x42, 0x06, 0x5a, 0x04, 0x2e, 0x2f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x1f, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2e, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x53, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0a, 0x2e, 0x70, + 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x12, 0x49, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x44, 0x69, 0x72, + 0x12, 0x20, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x45, 0x78, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x45, 0x78, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x42, 0x06, 0x5a, 0x04, 0x2e, 0x2f, 0x70, + 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -7057,7 +7016,7 @@ func file_pb_proto_rawDescGZIP() []byte { } var file_pb_proto_enumTypes = make([]protoimpl.EnumInfo, 9) -var file_pb_proto_msgTypes = make([]protoimpl.MessageInfo, 84) +var file_pb_proto_msgTypes = make([]protoimpl.MessageInfo, 83) var file_pb_proto_goTypes = []interface{}{ (DirectedEdge_Op)(0), // 0: pb.DirectedEdge.Op (Mutations_DropOp)(0), // 1: pb.Mutations.DropOp @@ -7119,47 +7078,47 @@ var file_pb_proto_goTypes = []interface{}{ (*TxnTimestamps)(nil), // 57: pb.TxnTimestamps (*PeerResponse)(nil), // 58: pb.PeerResponse (*RaftBatch)(nil), // 59: pb.RaftBatch - (*DrainModeRequest)(nil), // 60: pb.DrainModeRequest - (*TabletResponse)(nil), // 61: pb.TabletResponse - (*TabletRequest)(nil), // 62: pb.TabletRequest - (*SubscriptionRequest)(nil), // 63: pb.SubscriptionRequest - (*SubscriptionResponse)(nil), // 64: pb.SubscriptionResponse - (*Num)(nil), // 65: pb.Num - (*AssignedIds)(nil), // 66: pb.AssignedIds - (*RemoveNodeRequest)(nil), // 67: pb.RemoveNodeRequest - (*MoveTabletRequest)(nil), // 68: pb.MoveTabletRequest - (*SnapshotMeta)(nil), // 69: pb.SnapshotMeta - (*Status)(nil), // 70: pb.Status - (*BackupRequest)(nil), // 71: pb.BackupRequest - (*BackupResponse)(nil), // 72: pb.BackupResponse - (*DropOperation)(nil), // 73: pb.DropOperation - (*ExportRequest)(nil), // 74: pb.ExportRequest - (*ExportResponse)(nil), // 75: pb.ExportResponse - (*BackupKey)(nil), // 76: pb.BackupKey - (*BackupPostingList)(nil), // 77: pb.BackupPostingList - (*UpdateGraphQLSchemaRequest)(nil), // 78: pb.UpdateGraphQLSchemaRequest - (*UpdateGraphQLSchemaResponse)(nil), // 79: pb.UpdateGraphQLSchemaResponse - (*BulkMeta)(nil), // 80: pb.BulkMeta - (*DeleteNsRequest)(nil), // 81: pb.DeleteNsRequest - (*TaskStatusRequest)(nil), // 82: pb.TaskStatusRequest - (*TaskStatusResponse)(nil), // 83: pb.TaskStatusResponse - nil, // 84: pb.Result.VectorMetricsEntry - nil, // 85: pb.Group.MembersEntry - nil, // 86: pb.Group.TabletsEntry - nil, // 87: pb.ZeroProposal.SnapshotTsEntry - nil, // 88: pb.MembershipState.GroupsEntry - nil, // 89: pb.MembershipState.ZerosEntry - nil, // 90: pb.Metadata.PredHintsEntry - nil, // 91: pb.OracleDelta.GroupChecksumsEntry - nil, // 92: pb.BulkMeta.SchemaMapEntry - (*api.TxnContext)(nil), // 93: api.TxnContext - (*api.Facet)(nil), // 94: api.Facet - (*pb.KV)(nil), // 95: badgerpb4.KV - (*api.Payload)(nil), // 96: api.Payload - (*pb.Match)(nil), // 97: badgerpb4.Match - (*pb.KVList)(nil), // 98: badgerpb4.KVList - (*api_v2.StreamPDirRequest)(nil), // 99: api.v2.StreamPDirRequest - (*api_v2.StreamPDirResponse)(nil), // 100: api.v2.StreamPDirResponse + (*TabletResponse)(nil), // 60: pb.TabletResponse + (*TabletRequest)(nil), // 61: pb.TabletRequest + (*SubscriptionRequest)(nil), // 62: pb.SubscriptionRequest + (*SubscriptionResponse)(nil), // 63: pb.SubscriptionResponse + (*Num)(nil), // 64: pb.Num + (*AssignedIds)(nil), // 65: pb.AssignedIds + (*RemoveNodeRequest)(nil), // 66: pb.RemoveNodeRequest + (*MoveTabletRequest)(nil), // 67: pb.MoveTabletRequest + (*SnapshotMeta)(nil), // 68: pb.SnapshotMeta + (*Status)(nil), // 69: pb.Status + (*BackupRequest)(nil), // 70: pb.BackupRequest + (*BackupResponse)(nil), // 71: pb.BackupResponse + (*DropOperation)(nil), // 72: pb.DropOperation + (*ExportRequest)(nil), // 73: pb.ExportRequest + (*ExportResponse)(nil), // 74: pb.ExportResponse + (*BackupKey)(nil), // 75: pb.BackupKey + (*BackupPostingList)(nil), // 76: pb.BackupPostingList + (*UpdateGraphQLSchemaRequest)(nil), // 77: pb.UpdateGraphQLSchemaRequest + (*UpdateGraphQLSchemaResponse)(nil), // 78: pb.UpdateGraphQLSchemaResponse + (*BulkMeta)(nil), // 79: pb.BulkMeta + (*DeleteNsRequest)(nil), // 80: pb.DeleteNsRequest + (*TaskStatusRequest)(nil), // 81: pb.TaskStatusRequest + (*TaskStatusResponse)(nil), // 82: pb.TaskStatusResponse + nil, // 83: pb.Result.VectorMetricsEntry + nil, // 84: pb.Group.MembersEntry + nil, // 85: pb.Group.TabletsEntry + nil, // 86: pb.ZeroProposal.SnapshotTsEntry + nil, // 87: pb.MembershipState.GroupsEntry + nil, // 88: pb.MembershipState.ZerosEntry + nil, // 89: pb.Metadata.PredHintsEntry + nil, // 90: pb.OracleDelta.GroupChecksumsEntry + nil, // 91: pb.BulkMeta.SchemaMapEntry + (*api.TxnContext)(nil), // 92: api.TxnContext + (*api.Facet)(nil), // 93: api.Facet + (*pb.KV)(nil), // 94: badgerpb4.KV + (*api_v2.UpdateExtSnapshotStreamingStateRequest)(nil), // 95: api.v2.UpdateExtSnapshotStreamingStateRequest + (*api.Payload)(nil), // 96: api.Payload + (*pb.Match)(nil), // 97: badgerpb4.Match + (*pb.KVList)(nil), // 98: badgerpb4.KVList + (*api_v2.StreamExtSnapshotRequest)(nil), // 99: api.v2.StreamExtSnapshotRequest + (*api_v2.StreamExtSnapshotResponse)(nil), // 100: api.v2.StreamExtSnapshotResponse } var file_pb_proto_depIdxs = []int32{ 3, // 0: pb.TaskValue.val_type:type_name -> pb.Posting.ValType @@ -7172,52 +7131,52 @@ var file_pb_proto_depIdxs = []int32{ 13, // 7: pb.Result.value_matrix:type_name -> pb.ValueList 43, // 8: pb.Result.facet_matrix:type_name -> pb.FacetsList 14, // 9: pb.Result.lang_matrix:type_name -> pb.LangList - 84, // 10: pb.Result.vector_metrics:type_name -> pb.Result.VectorMetricsEntry + 83, // 10: pb.Result.vector_metrics:type_name -> pb.Result.VectorMetricsEntry 16, // 11: pb.SortMessage.order:type_name -> pb.Order 9, // 12: pb.SortMessage.uid_matrix:type_name -> pb.List 9, // 13: pb.SortResult.uid_matrix:type_name -> pb.List - 85, // 14: pb.Group.members:type_name -> pb.Group.MembersEntry - 86, // 15: pb.Group.tablets:type_name -> pb.Group.TabletsEntry - 87, // 16: pb.ZeroProposal.snapshot_ts:type_name -> pb.ZeroProposal.SnapshotTsEntry + 84, // 14: pb.Group.members:type_name -> pb.Group.MembersEntry + 85, // 15: pb.Group.tablets:type_name -> pb.Group.TabletsEntry + 86, // 16: pb.ZeroProposal.snapshot_ts:type_name -> pb.ZeroProposal.SnapshotTsEntry 20, // 17: pb.ZeroProposal.member:type_name -> pb.Member 26, // 18: pb.ZeroProposal.tablet:type_name -> pb.Tablet - 93, // 19: pb.ZeroProposal.txn:type_name -> api.TxnContext + 92, // 19: pb.ZeroProposal.txn:type_name -> api.TxnContext 31, // 20: pb.ZeroProposal.snapshot:type_name -> pb.ZeroSnapshot - 81, // 21: pb.ZeroProposal.delete_ns:type_name -> pb.DeleteNsRequest + 80, // 21: pb.ZeroProposal.delete_ns:type_name -> pb.DeleteNsRequest 26, // 22: pb.ZeroProposal.tablets:type_name -> pb.Tablet - 88, // 23: pb.MembershipState.groups:type_name -> pb.MembershipState.GroupsEntry - 89, // 24: pb.MembershipState.zeros:type_name -> pb.MembershipState.ZerosEntry + 87, // 23: pb.MembershipState.groups:type_name -> pb.MembershipState.GroupsEntry + 88, // 24: pb.MembershipState.zeros:type_name -> pb.MembershipState.ZerosEntry 20, // 25: pb.MembershipState.removed:type_name -> pb.Member 20, // 26: pb.ConnectionState.member:type_name -> pb.Member 23, // 27: pb.ConnectionState.state:type_name -> pb.MembershipState 3, // 28: pb.DirectedEdge.value_type:type_name -> pb.Posting.ValType 0, // 29: pb.DirectedEdge.op:type_name -> pb.DirectedEdge.Op - 94, // 30: pb.DirectedEdge.facets:type_name -> api.Facet + 93, // 30: pb.DirectedEdge.facets:type_name -> api.Facet 27, // 31: pb.Mutations.edges:type_name -> pb.DirectedEdge 49, // 32: pb.Mutations.schema:type_name -> pb.SchemaUpdate 52, // 33: pb.Mutations.types:type_name -> pb.TypeUpdate 1, // 34: pb.Mutations.drop_op:type_name -> pb.Mutations.DropOp 29, // 35: pb.Mutations.metadata:type_name -> pb.Metadata - 90, // 36: pb.Metadata.pred_hints:type_name -> pb.Metadata.PredHintsEntry + 89, // 36: pb.Metadata.pred_hints:type_name -> pb.Metadata.PredHintsEntry 19, // 37: pb.Snapshot.context:type_name -> pb.RaftContext 23, // 38: pb.ZeroSnapshot.state:type_name -> pb.MembershipState 28, // 39: pb.Proposal.mutations:type_name -> pb.Mutations - 95, // 40: pb.Proposal.kv:type_name -> badgerpb4.KV + 94, // 40: pb.Proposal.kv:type_name -> badgerpb4.KV 23, // 41: pb.Proposal.state:type_name -> pb.MembershipState 56, // 42: pb.Proposal.delta:type_name -> pb.OracleDelta 30, // 43: pb.Proposal.snapshot:type_name -> pb.Snapshot 32, // 44: pb.Proposal.restore:type_name -> pb.RestoreRequest 34, // 45: pb.Proposal.cdc_state:type_name -> pb.CDCState - 81, // 46: pb.Proposal.delete_ns:type_name -> pb.DeleteNsRequest - 60, // 47: pb.Proposal.drainmode:type_name -> pb.DrainModeRequest + 80, // 46: pb.Proposal.delete_ns:type_name -> pb.DeleteNsRequest + 95, // 47: pb.Proposal.update_ext_snapshot_streaming_state:type_name -> api.v2.UpdateExtSnapshotStreamingStateRequest 3, // 48: pb.Posting.val_type:type_name -> pb.Posting.ValType 4, // 49: pb.Posting.posting_type:type_name -> pb.Posting.PostingType - 94, // 50: pb.Posting.facets:type_name -> api.Facet + 93, // 50: pb.Posting.facets:type_name -> api.Facet 37, // 51: pb.UidPack.blocks:type_name -> pb.UidBlock 38, // 52: pb.PostingList.pack:type_name -> pb.UidPack 36, // 53: pb.PostingList.postings:type_name -> pb.Posting 40, // 54: pb.FacetParams.param:type_name -> pb.FacetParam - 94, // 55: pb.Facets.facets:type_name -> api.Facet + 93, // 55: pb.Facets.facets:type_name -> api.Facet 42, // 56: pb.FacetsList.facets_list:type_name -> pb.Facets 45, // 57: pb.FilterTree.children:type_name -> pb.FilterTree 44, // 58: pb.FilterTree.func:type_name -> pb.Function @@ -7229,7 +7188,7 @@ var file_pb_proto_depIdxs = []int32{ 51, // 64: pb.VectorIndexSpec.options:type_name -> pb.OptionPair 49, // 65: pb.TypeUpdate.fields:type_name -> pb.SchemaUpdate 55, // 66: pb.OracleDelta.txns:type_name -> pb.TxnStatus - 91, // 67: pb.OracleDelta.group_checksums:type_name -> pb.OracleDelta.GroupChecksumsEntry + 90, // 67: pb.OracleDelta.group_checksums:type_name -> pb.OracleDelta.GroupChecksumsEntry 19, // 68: pb.RaftBatch.context:type_name -> pb.RaftContext 96, // 69: pb.RaftBatch.payload:type_name -> api.Payload 26, // 70: pb.TabletResponse.tablets:type_name -> pb.Tablet @@ -7237,13 +7196,13 @@ var file_pb_proto_depIdxs = []int32{ 97, // 72: pb.SubscriptionRequest.matches:type_name -> badgerpb4.Match 98, // 73: pb.SubscriptionResponse.kvs:type_name -> badgerpb4.KVList 6, // 74: pb.Num.type:type_name -> pb.Num.leaseType - 73, // 75: pb.BackupResponse.drop_operations:type_name -> pb.DropOperation + 72, // 75: pb.BackupResponse.drop_operations:type_name -> pb.DropOperation 7, // 76: pb.DropOperation.drop_op:type_name -> pb.DropOperation.DropOp 8, // 77: pb.BackupKey.type:type_name -> pb.BackupKey.KeyType 36, // 78: pb.BackupPostingList.postings:type_name -> pb.Posting 49, // 79: pb.UpdateGraphQLSchemaRequest.dgraph_preds:type_name -> pb.SchemaUpdate 52, // 80: pb.UpdateGraphQLSchemaRequest.dgraph_types:type_name -> pb.TypeUpdate - 92, // 81: pb.BulkMeta.schema_map:type_name -> pb.BulkMeta.SchemaMapEntry + 91, // 81: pb.BulkMeta.schema_map:type_name -> pb.BulkMeta.SchemaMapEntry 52, // 82: pb.BulkMeta.types:type_name -> pb.TypeUpdate 20, // 83: pb.Group.MembersEntry.value:type_name -> pb.Member 26, // 84: pb.Group.TabletsEntry.value:type_name -> pb.Tablet @@ -7260,30 +7219,30 @@ var file_pb_proto_depIdxs = []int32{ 96, // 95: pb.Zero.StreamMembership:input_type -> api.Payload 96, // 96: pb.Zero.Oracle:input_type -> api.Payload 26, // 97: pb.Zero.ShouldServe:input_type -> pb.Tablet - 62, // 98: pb.Zero.Inform:input_type -> pb.TabletRequest - 65, // 99: pb.Zero.AssignIds:input_type -> pb.Num - 65, // 100: pb.Zero.Timestamps:input_type -> pb.Num - 93, // 101: pb.Zero.CommitOrAbort:input_type -> api.TxnContext + 61, // 98: pb.Zero.Inform:input_type -> pb.TabletRequest + 64, // 99: pb.Zero.AssignIds:input_type -> pb.Num + 64, // 100: pb.Zero.Timestamps:input_type -> pb.Num + 92, // 101: pb.Zero.CommitOrAbort:input_type -> api.TxnContext 57, // 102: pb.Zero.TryAbort:input_type -> pb.TxnTimestamps - 81, // 103: pb.Zero.DeleteNamespace:input_type -> pb.DeleteNsRequest - 67, // 104: pb.Zero.RemoveNode:input_type -> pb.RemoveNodeRequest - 68, // 105: pb.Zero.MoveTablet:input_type -> pb.MoveTabletRequest + 80, // 103: pb.Zero.DeleteNamespace:input_type -> pb.DeleteNsRequest + 66, // 104: pb.Zero.RemoveNode:input_type -> pb.RemoveNodeRequest + 67, // 105: pb.Zero.MoveTablet:input_type -> pb.MoveTabletRequest 28, // 106: pb.Worker.Mutate:input_type -> pb.Mutations 12, // 107: pb.Worker.ServeTask:input_type -> pb.Query 30, // 108: pb.Worker.StreamSnapshot:input_type -> pb.Snapshot 17, // 109: pb.Worker.Sort:input_type -> pb.SortMessage 46, // 110: pb.Worker.Schema:input_type -> pb.SchemaRequest - 71, // 111: pb.Worker.Backup:input_type -> pb.BackupRequest + 70, // 111: pb.Worker.Backup:input_type -> pb.BackupRequest 32, // 112: pb.Worker.Restore:input_type -> pb.RestoreRequest - 74, // 113: pb.Worker.Export:input_type -> pb.ExportRequest + 73, // 113: pb.Worker.Export:input_type -> pb.ExportRequest 35, // 114: pb.Worker.ReceivePredicate:input_type -> pb.KVS 54, // 115: pb.Worker.MovePredicate:input_type -> pb.MovePredicatePayload - 63, // 116: pb.Worker.Subscribe:input_type -> pb.SubscriptionRequest - 78, // 117: pb.Worker.UpdateGraphQLSchema:input_type -> pb.UpdateGraphQLSchemaRequest - 81, // 118: pb.Worker.DeleteNamespace:input_type -> pb.DeleteNsRequest - 82, // 119: pb.Worker.TaskStatus:input_type -> pb.TaskStatusRequest - 60, // 120: pb.Worker.ApplyDrainmode:input_type -> pb.DrainModeRequest - 99, // 121: pb.Worker.InternalStreamPDir:input_type -> api.v2.StreamPDirRequest + 62, // 116: pb.Worker.Subscribe:input_type -> pb.SubscriptionRequest + 77, // 117: pb.Worker.UpdateGraphQLSchema:input_type -> pb.UpdateGraphQLSchemaRequest + 80, // 118: pb.Worker.DeleteNamespace:input_type -> pb.DeleteNsRequest + 81, // 119: pb.Worker.TaskStatus:input_type -> pb.TaskStatusRequest + 95, // 120: pb.Worker.UpdateExtSnapshotStreamingState:input_type -> api.v2.UpdateExtSnapshotStreamingStateRequest + 99, // 121: pb.Worker.InternalStreamPDir:input_type -> api.v2.StreamExtSnapshotRequest 25, // 122: pb.Raft.Heartbeat:output_type -> pb.HealthInfo 96, // 123: pb.Raft.RaftMessage:output_type -> api.Payload 96, // 124: pb.Raft.JoinCluster:output_type -> api.Payload @@ -7293,30 +7252,30 @@ var file_pb_proto_depIdxs = []int32{ 23, // 128: pb.Zero.StreamMembership:output_type -> pb.MembershipState 56, // 129: pb.Zero.Oracle:output_type -> pb.OracleDelta 26, // 130: pb.Zero.ShouldServe:output_type -> pb.Tablet - 61, // 131: pb.Zero.Inform:output_type -> pb.TabletResponse - 66, // 132: pb.Zero.AssignIds:output_type -> pb.AssignedIds - 66, // 133: pb.Zero.Timestamps:output_type -> pb.AssignedIds - 93, // 134: pb.Zero.CommitOrAbort:output_type -> api.TxnContext + 60, // 131: pb.Zero.Inform:output_type -> pb.TabletResponse + 65, // 132: pb.Zero.AssignIds:output_type -> pb.AssignedIds + 65, // 133: pb.Zero.Timestamps:output_type -> pb.AssignedIds + 92, // 134: pb.Zero.CommitOrAbort:output_type -> api.TxnContext 56, // 135: pb.Zero.TryAbort:output_type -> pb.OracleDelta - 70, // 136: pb.Zero.DeleteNamespace:output_type -> pb.Status - 70, // 137: pb.Zero.RemoveNode:output_type -> pb.Status - 70, // 138: pb.Zero.MoveTablet:output_type -> pb.Status - 93, // 139: pb.Worker.Mutate:output_type -> api.TxnContext + 69, // 136: pb.Zero.DeleteNamespace:output_type -> pb.Status + 69, // 137: pb.Zero.RemoveNode:output_type -> pb.Status + 69, // 138: pb.Zero.MoveTablet:output_type -> pb.Status + 92, // 139: pb.Worker.Mutate:output_type -> api.TxnContext 15, // 140: pb.Worker.ServeTask:output_type -> pb.Result 35, // 141: pb.Worker.StreamSnapshot:output_type -> pb.KVS 18, // 142: pb.Worker.Sort:output_type -> pb.SortResult 48, // 143: pb.Worker.Schema:output_type -> pb.SchemaResult - 72, // 144: pb.Worker.Backup:output_type -> pb.BackupResponse - 70, // 145: pb.Worker.Restore:output_type -> pb.Status - 75, // 146: pb.Worker.Export:output_type -> pb.ExportResponse + 71, // 144: pb.Worker.Backup:output_type -> pb.BackupResponse + 69, // 145: pb.Worker.Restore:output_type -> pb.Status + 74, // 146: pb.Worker.Export:output_type -> pb.ExportResponse 96, // 147: pb.Worker.ReceivePredicate:output_type -> api.Payload 96, // 148: pb.Worker.MovePredicate:output_type -> api.Payload 98, // 149: pb.Worker.Subscribe:output_type -> badgerpb4.KVList - 79, // 150: pb.Worker.UpdateGraphQLSchema:output_type -> pb.UpdateGraphQLSchemaResponse - 70, // 151: pb.Worker.DeleteNamespace:output_type -> pb.Status - 83, // 152: pb.Worker.TaskStatus:output_type -> pb.TaskStatusResponse - 70, // 153: pb.Worker.ApplyDrainmode:output_type -> pb.Status - 100, // 154: pb.Worker.InternalStreamPDir:output_type -> api.v2.StreamPDirResponse + 78, // 150: pb.Worker.UpdateGraphQLSchema:output_type -> pb.UpdateGraphQLSchemaResponse + 69, // 151: pb.Worker.DeleteNamespace:output_type -> pb.Status + 82, // 152: pb.Worker.TaskStatus:output_type -> pb.TaskStatusResponse + 69, // 153: pb.Worker.UpdateExtSnapshotStreamingState:output_type -> pb.Status + 100, // 154: pb.Worker.InternalStreamPDir:output_type -> api.v2.StreamExtSnapshotResponse 122, // [122:155] is the sub-list for method output_type 89, // [89:122] is the sub-list for method input_type 89, // [89:89] is the sub-list for extension type_name @@ -7943,18 +7902,6 @@ func file_pb_proto_init() { } } file_pb_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DrainModeRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_pb_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletResponse); i { case 0: return &v.state @@ -7966,7 +7913,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletRequest); i { case 0: return &v.state @@ -7978,7 +7925,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscriptionRequest); i { case 0: return &v.state @@ -7990,7 +7937,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscriptionResponse); i { case 0: return &v.state @@ -8002,7 +7949,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Num); i { case 0: return &v.state @@ -8014,7 +7961,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AssignedIds); i { case 0: return &v.state @@ -8026,7 +7973,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveNodeRequest); i { case 0: return &v.state @@ -8038,7 +7985,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTabletRequest); i { case 0: return &v.state @@ -8050,7 +7997,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SnapshotMeta); i { case 0: return &v.state @@ -8062,7 +8009,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Status); i { case 0: return &v.state @@ -8074,7 +8021,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupRequest); i { case 0: return &v.state @@ -8086,7 +8033,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupResponse); i { case 0: return &v.state @@ -8098,7 +8045,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropOperation); i { case 0: return &v.state @@ -8110,7 +8057,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExportRequest); i { case 0: return &v.state @@ -8122,7 +8069,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExportResponse); i { case 0: return &v.state @@ -8134,7 +8081,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupKey); i { case 0: return &v.state @@ -8146,7 +8093,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupPostingList); i { case 0: return &v.state @@ -8158,7 +8105,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateGraphQLSchemaRequest); i { case 0: return &v.state @@ -8170,7 +8117,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateGraphQLSchemaResponse); i { case 0: return &v.state @@ -8182,7 +8129,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BulkMeta); i { case 0: return &v.state @@ -8194,7 +8141,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteNsRequest); i { case 0: return &v.state @@ -8206,7 +8153,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TaskStatusRequest); i { case 0: return &v.state @@ -8218,7 +8165,7 @@ func file_pb_proto_init() { return nil } } - file_pb_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { + file_pb_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TaskStatusResponse); i { case 0: return &v.state @@ -8237,7 +8184,7 @@ func file_pb_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_pb_proto_rawDesc, NumEnums: 9, - NumMessages: 84, + NumMessages: 83, NumExtensions: 0, NumServices: 3, }, diff --git a/protos/pb/pb_grpc.pb.go b/protos/pb/pb_grpc.pb.go index 78799b7b68b..3caa44ffced 100644 --- a/protos/pb/pb_grpc.pb.go +++ b/protos/pb/pb_grpc.pb.go @@ -885,22 +885,22 @@ var Zero_ServiceDesc = grpc.ServiceDesc{ } const ( - Worker_Mutate_FullMethodName = "/pb.Worker/Mutate" - Worker_ServeTask_FullMethodName = "/pb.Worker/ServeTask" - Worker_StreamSnapshot_FullMethodName = "/pb.Worker/StreamSnapshot" - Worker_Sort_FullMethodName = "/pb.Worker/Sort" - Worker_Schema_FullMethodName = "/pb.Worker/Schema" - Worker_Backup_FullMethodName = "/pb.Worker/Backup" - Worker_Restore_FullMethodName = "/pb.Worker/Restore" - Worker_Export_FullMethodName = "/pb.Worker/Export" - Worker_ReceivePredicate_FullMethodName = "/pb.Worker/ReceivePredicate" - Worker_MovePredicate_FullMethodName = "/pb.Worker/MovePredicate" - Worker_Subscribe_FullMethodName = "/pb.Worker/Subscribe" - Worker_UpdateGraphQLSchema_FullMethodName = "/pb.Worker/UpdateGraphQLSchema" - Worker_DeleteNamespace_FullMethodName = "/pb.Worker/DeleteNamespace" - Worker_TaskStatus_FullMethodName = "/pb.Worker/TaskStatus" - Worker_ApplyDrainmode_FullMethodName = "/pb.Worker/ApplyDrainmode" - Worker_InternalStreamPDir_FullMethodName = "/pb.Worker/InternalStreamPDir" + Worker_Mutate_FullMethodName = "/pb.Worker/Mutate" + Worker_ServeTask_FullMethodName = "/pb.Worker/ServeTask" + Worker_StreamSnapshot_FullMethodName = "/pb.Worker/StreamSnapshot" + Worker_Sort_FullMethodName = "/pb.Worker/Sort" + Worker_Schema_FullMethodName = "/pb.Worker/Schema" + Worker_Backup_FullMethodName = "/pb.Worker/Backup" + Worker_Restore_FullMethodName = "/pb.Worker/Restore" + Worker_Export_FullMethodName = "/pb.Worker/Export" + Worker_ReceivePredicate_FullMethodName = "/pb.Worker/ReceivePredicate" + Worker_MovePredicate_FullMethodName = "/pb.Worker/MovePredicate" + Worker_Subscribe_FullMethodName = "/pb.Worker/Subscribe" + Worker_UpdateGraphQLSchema_FullMethodName = "/pb.Worker/UpdateGraphQLSchema" + Worker_DeleteNamespace_FullMethodName = "/pb.Worker/DeleteNamespace" + Worker_TaskStatus_FullMethodName = "/pb.Worker/TaskStatus" + Worker_UpdateExtSnapshotStreamingState_FullMethodName = "/pb.Worker/UpdateExtSnapshotStreamingState" + Worker_InternalStreamPDir_FullMethodName = "/pb.Worker/InternalStreamPDir" ) // WorkerClient is the client API for Worker service. @@ -922,7 +922,7 @@ type WorkerClient interface { UpdateGraphQLSchema(ctx context.Context, in *UpdateGraphQLSchemaRequest, opts ...grpc.CallOption) (*UpdateGraphQLSchemaResponse, error) DeleteNamespace(ctx context.Context, in *DeleteNsRequest, opts ...grpc.CallOption) (*Status, error) TaskStatus(ctx context.Context, in *TaskStatusRequest, opts ...grpc.CallOption) (*TaskStatusResponse, error) - ApplyDrainmode(ctx context.Context, in *DrainModeRequest, opts ...grpc.CallOption) (*Status, error) + UpdateExtSnapshotStreamingState(ctx context.Context, in *api_v2.UpdateExtSnapshotStreamingStateRequest, opts ...grpc.CallOption) (*Status, error) InternalStreamPDir(ctx context.Context, opts ...grpc.CallOption) (Worker_InternalStreamPDirClient, error) } @@ -1130,9 +1130,9 @@ func (c *workerClient) TaskStatus(ctx context.Context, in *TaskStatusRequest, op return out, nil } -func (c *workerClient) ApplyDrainmode(ctx context.Context, in *DrainModeRequest, opts ...grpc.CallOption) (*Status, error) { +func (c *workerClient) UpdateExtSnapshotStreamingState(ctx context.Context, in *api_v2.UpdateExtSnapshotStreamingStateRequest, opts ...grpc.CallOption) (*Status, error) { out := new(Status) - err := c.cc.Invoke(ctx, Worker_ApplyDrainmode_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, Worker_UpdateExtSnapshotStreamingState_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -1149,8 +1149,8 @@ func (c *workerClient) InternalStreamPDir(ctx context.Context, opts ...grpc.Call } type Worker_InternalStreamPDirClient interface { - Send(*api_v2.StreamPDirRequest) error - CloseAndRecv() (*api_v2.StreamPDirResponse, error) + Send(*api_v2.StreamExtSnapshotRequest) error + CloseAndRecv() (*api_v2.StreamExtSnapshotResponse, error) grpc.ClientStream } @@ -1158,15 +1158,15 @@ type workerInternalStreamPDirClient struct { grpc.ClientStream } -func (x *workerInternalStreamPDirClient) Send(m *api_v2.StreamPDirRequest) error { +func (x *workerInternalStreamPDirClient) Send(m *api_v2.StreamExtSnapshotRequest) error { return x.ClientStream.SendMsg(m) } -func (x *workerInternalStreamPDirClient) CloseAndRecv() (*api_v2.StreamPDirResponse, error) { +func (x *workerInternalStreamPDirClient) CloseAndRecv() (*api_v2.StreamExtSnapshotResponse, error) { if err := x.ClientStream.CloseSend(); err != nil { return nil, err } - m := new(api_v2.StreamPDirResponse) + m := new(api_v2.StreamExtSnapshotResponse) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } @@ -1192,7 +1192,7 @@ type WorkerServer interface { UpdateGraphQLSchema(context.Context, *UpdateGraphQLSchemaRequest) (*UpdateGraphQLSchemaResponse, error) DeleteNamespace(context.Context, *DeleteNsRequest) (*Status, error) TaskStatus(context.Context, *TaskStatusRequest) (*TaskStatusResponse, error) - ApplyDrainmode(context.Context, *DrainModeRequest) (*Status, error) + UpdateExtSnapshotStreamingState(context.Context, *api_v2.UpdateExtSnapshotStreamingStateRequest) (*Status, error) InternalStreamPDir(Worker_InternalStreamPDirServer) error mustEmbedUnimplementedWorkerServer() } @@ -1243,8 +1243,8 @@ func (UnimplementedWorkerServer) DeleteNamespace(context.Context, *DeleteNsReque func (UnimplementedWorkerServer) TaskStatus(context.Context, *TaskStatusRequest) (*TaskStatusResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TaskStatus not implemented") } -func (UnimplementedWorkerServer) ApplyDrainmode(context.Context, *DrainModeRequest) (*Status, error) { - return nil, status.Errorf(codes.Unimplemented, "method ApplyDrainmode not implemented") +func (UnimplementedWorkerServer) UpdateExtSnapshotStreamingState(context.Context, *api_v2.UpdateExtSnapshotStreamingStateRequest) (*Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateExtSnapshotStreamingState not implemented") } func (UnimplementedWorkerServer) InternalStreamPDir(Worker_InternalStreamPDirServer) error { return status.Errorf(codes.Unimplemented, "method InternalStreamPDir not implemented") @@ -1533,20 +1533,20 @@ func _Worker_TaskStatus_Handler(srv interface{}, ctx context.Context, dec func(i return interceptor(ctx, in, info, handler) } -func _Worker_ApplyDrainmode_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(DrainModeRequest) +func _Worker_UpdateExtSnapshotStreamingState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(api_v2.UpdateExtSnapshotStreamingStateRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(WorkerServer).ApplyDrainmode(ctx, in) + return srv.(WorkerServer).UpdateExtSnapshotStreamingState(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: Worker_ApplyDrainmode_FullMethodName, + FullMethod: Worker_UpdateExtSnapshotStreamingState_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(WorkerServer).ApplyDrainmode(ctx, req.(*DrainModeRequest)) + return srv.(WorkerServer).UpdateExtSnapshotStreamingState(ctx, req.(*api_v2.UpdateExtSnapshotStreamingStateRequest)) } return interceptor(ctx, in, info, handler) } @@ -1556,8 +1556,8 @@ func _Worker_InternalStreamPDir_Handler(srv interface{}, stream grpc.ServerStrea } type Worker_InternalStreamPDirServer interface { - SendAndClose(*api_v2.StreamPDirResponse) error - Recv() (*api_v2.StreamPDirRequest, error) + SendAndClose(*api_v2.StreamExtSnapshotResponse) error + Recv() (*api_v2.StreamExtSnapshotRequest, error) grpc.ServerStream } @@ -1565,12 +1565,12 @@ type workerInternalStreamPDirServer struct { grpc.ServerStream } -func (x *workerInternalStreamPDirServer) SendAndClose(m *api_v2.StreamPDirResponse) error { +func (x *workerInternalStreamPDirServer) SendAndClose(m *api_v2.StreamExtSnapshotResponse) error { return x.ServerStream.SendMsg(m) } -func (x *workerInternalStreamPDirServer) Recv() (*api_v2.StreamPDirRequest, error) { - m := new(api_v2.StreamPDirRequest) +func (x *workerInternalStreamPDirServer) Recv() (*api_v2.StreamExtSnapshotRequest, error) { + m := new(api_v2.StreamExtSnapshotRequest) if err := x.ServerStream.RecvMsg(m); err != nil { return nil, err } @@ -1629,8 +1629,8 @@ var Worker_ServiceDesc = grpc.ServiceDesc{ Handler: _Worker_TaskStatus_Handler, }, { - MethodName: "ApplyDrainmode", - Handler: _Worker_ApplyDrainmode_Handler, + MethodName: "UpdateExtSnapshotStreamingState", + Handler: _Worker_UpdateExtSnapshotStreamingState_Handler, }, }, Streams: []grpc.StreamDesc{ diff --git a/worker/draft.go b/worker/draft.go index 80aa864ea5e..ae2f17d396a 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -689,8 +689,70 @@ func (n *node) applyCommitted(proposal *pb.Proposal, key uint64) error { attribute.Int64("key", int64(key)))) return n.commitOrAbort(key, proposal.Delta) - case proposal.Drainmode != nil: - x.UpdateDrainingMode(proposal.Drainmode.State) + case proposal.UpdateExtSnapshotStreamingState != nil: + // Handle UpdateExtSnapshotStreamingState proposal with two distinct scenarios: + // 1. Node was offline during streaming and is now rejoining so this we are figuring out using + // x.ExtSnapshotStreamingState if that is true then we can say node is rejoined because + // we are disabling it after completion of streaming: + // - We request and populate snapshot from the group leader + // - After receiving the snapshot, we perform post-stream processing to ensure data consistency + // + // 2. Node was online during streaming: + // - If the node was online during streaming, we only need to disable drain mode + // - No additional data synchronization is required since the node was already receiving updates + // + // Drop Data Handling: + // - When DropData is true, we clear all data from the node + // - This is used as a safety measure when: + // - Something went wrong with other groups/nodes + // - The node needs to be brought back in sync with the cluster + // - Ensures the node reaches a clean, consistent state + + if proposal.UpdateExtSnapshotStreamingState.Start { + x.ExtSnapshotStreamingState(true) + return nil + } + + if proposal.UpdateExtSnapshotStreamingState.DropData { + // When DropData is true, it implies that there was some issue with import (e.g. some group + // may have started streaming but not finished or streaming never started because p directory + // was not there). We can't be certain if we have disabled x.UpdateExtSnapshotStreamingState or + // not, so we are disabling it here: + x.ExtSnapshotStreamingState(false) + err := posting.DeleteAll() + if err != nil { + glog.Errorf("[import] failed to delete all data: %v", err) + return err + } + } + + if proposal.UpdateExtSnapshotStreamingState.Finish && x.IsExtSnapshotStreamingStateTrue() { + lastApplied := n.Applied.LastIndex() + glog.Infof("[import] requesting snapshot from leader") + snap := &pb.Snapshot{ + Context: n.RaftContext, + Index: lastApplied, + ReadTs: State.GetTimestamp(false), + } + pl := groups().Leader(n.gid) + if pl == nil { + glog.Errorf("[import] failed to get connection pool for group %d", n.gid) + return errors.Errorf("failed to get connection pool for group %d", n.gid) + } + // Request and populate snapshot + if err := n.populateSnapshot(snap, pl); err != nil { + glog.Errorf("[import] failed to populate snapshot for rejoining node: %v", err) + return errors.Wrapf(err, "failed to populate snapshot for rejoining node") + } + + if err := postStreamProcessing(ctx); err != nil { + glog.Errorf("[import] failed to post stream processing for rejoining node: %v", err) + return errors.Wrapf(err, "failed to post stream processing for rejoining node") + } + glog.Infof("[import] successfully rejoined node") + x.ExtSnapshotStreamingState(false) + } + return nil case proposal.Snapshot != nil: diff --git a/worker/import.go b/worker/import.go index 8730a1af9c7..488256a40de 100644 --- a/worker/import.go +++ b/worker/import.go @@ -9,54 +9,92 @@ import ( "context" "fmt" "io" + "math" + "sync" + "github.com/dgraph-io/badger/v4" apiv2 "github.com/dgraph-io/dgo/v250/protos/api.v2" "github.com/dgraph-io/ristretto/v2/z" "github.com/hypermodeinc/dgraph/v25/conn" "github.com/hypermodeinc/dgraph/v25/posting" "github.com/hypermodeinc/dgraph/v25/protos/pb" "github.com/hypermodeinc/dgraph/v25/schema" + "github.com/hypermodeinc/dgraph/v25/x" + "golang.org/x/sync/errgroup" "github.com/dustin/go-humanize" "github.com/golang/glog" "github.com/pkg/errors" - "google.golang.org/grpc" ) -// streamProcessor defines the common interface for stream processing -type streamProcessor interface { - SendAndClose(*apiv2.StreamPDirResponse) error - Recv() (*apiv2.StreamPDirRequest, error) - grpc.ServerStream +type pubSub struct { + subscribers []chan *apiv2.StreamExtSnapshotRequest + lock sync.RWMutex } -func ProposeDrain(ctx context.Context, drainMode *pb.DrainModeRequest) ([]uint32, error) { +// Subscribe returns a new channel to receive published messages +func (ps *pubSub) subscribe() <-chan *apiv2.StreamExtSnapshotRequest { + ch := make(chan *apiv2.StreamExtSnapshotRequest, 10) // buffered so slow consumers don't block + ps.lock.Lock() + defer ps.lock.Unlock() + ps.subscribers = append(ps.subscribers, ch) + return ch +} + +func (ps *pubSub) publish(msg *apiv2.StreamExtSnapshotRequest) { + ps.lock.RLock() + defer ps.lock.RUnlock() + for _, ch := range ps.subscribers { + select { + case ch <- msg: + default: + // drop message if subscriber is slow + fmt.Println("Dropping message for slow subscriber") + } + } +} + +func (ps *pubSub) close() { + ps.lock.Lock() + defer ps.lock.Unlock() + for _, ch := range ps.subscribers { + close(ch) + } + ps.subscribers = nil +} + +func ProposeDrain(ctx context.Context, drainMode *apiv2.UpdateExtSnapshotStreamingStateRequest) ([]uint32, error) { memState := GetMembershipState() currentGroups := make([]uint32, 0) for gid := range memState.GetGroups() { currentGroups = append(currentGroups, gid) } + updateExtSnapshotStreamingState := &apiv2.UpdateExtSnapshotStreamingStateRequest{ + Finish: drainMode.Finish, + Start: drainMode.Start, + DropData: drainMode.DropData, + } for _, gid := range currentGroups { if groups().ServesGroup(gid) && groups().Node.AmLeader() { - if _, err := (&grpcWorker{}).ApplyDrainmode(ctx, drainMode); err != nil { + if _, err := (&grpcWorker{}).UpdateExtSnapshotStreamingState(ctx, updateExtSnapshotStreamingState); err != nil { return nil, err } continue } - glog.Infof("[import] Connecting to the leader of the group [%v] from alpha addr [%v]", gid, groups().Node.MyAddr) + glog.Infof("[import:apply-drainmode] Connecting to the leader of the group [%v] from alpha addr [%v]", gid, groups().Node.MyAddr) pl := groups().Leader(gid) if pl == nil { - glog.Errorf("[import] unable to connect to the leader of group [%v]", gid) + glog.Errorf("[import:apply-drainmode] unable to connect to the leader of group [%v]", gid) return nil, fmt.Errorf("unable to connect to the leader of group [%v] : %v", gid, conn.ErrNoConnection) } con := pl.Get() c := pb.NewWorkerClient(con) - glog.Infof("[import] Successfully connected to leader of group [%v]", gid) + glog.Infof("[import:apply-drainmode] Successfully connected to leader of group [%v]", gid) - if _, err := c.ApplyDrainmode(ctx, drainMode); err != nil { - glog.Errorf("[import] unable to apply drainmode : %v", err) + if _, err := c.UpdateExtSnapshotStreamingState(ctx, updateExtSnapshotStreamingState); err != nil { + glog.Errorf("[import:apply-drainmode] unable to apply drainmode : %v", err) return nil, err } } @@ -70,7 +108,8 @@ func ProposeDrain(ctx context.Context, drainMode *pb.DrainModeRequest) ([]uint32 // with the leader of that group and streams data to it. The function returns an error if // there are any issues in the process, such as a broken connection or failure to establish // a stream with the leader. -func InStream(stream apiv2.Dgraph_StreamPDirServer) error { +func InStream(stream apiv2.Dgraph_StreamExtSnapshotServer) error { + req, err := stream.Recv() if err != nil { return fmt.Errorf("failed to receive initial stream message: %v", err) @@ -78,9 +117,11 @@ func InStream(stream apiv2.Dgraph_StreamPDirServer) error { groupId := req.GroupId if groupId == groups().Node.gid { - return flushKvs(stream) + glog.Infof("[import] Streaming P dir to current Group [%v]", groupId) + return streamInGroup(stream, true) } + glog.Infof("[import] Streaming P dir to other Group [%v]", groupId) pl := groups().Leader(groupId) if pl == nil { glog.Errorf("[import] Unable to connect to the leader of group [%v]", groupId) @@ -94,27 +135,34 @@ func InStream(stream apiv2.Dgraph_StreamPDirServer) error { return fmt.Errorf("failed to establish stream with leader: %v", err) } - return pipeTwoStream(stream, alphaStream) + glog.Infof("[import] sending forward true to leader of group [%v]", groupId) + forwardReq := &apiv2.StreamExtSnapshotRequest{Forward: true} + if err := alphaStream.Send(forwardReq); err != nil { + return fmt.Errorf("failed to send forward request: %w", err) + } + + return pipeTwoStream(stream, alphaStream, groupId) } -func pipeTwoStream(in apiv2.Dgraph_StreamPDirServer, out pb.Worker_InternalStreamPDirClient) error { - buffer := make(chan *apiv2.StreamPDirRequest, 10) +func pipeTwoStream(in apiv2.Dgraph_StreamExtSnapshotServer, out pb.Worker_InternalStreamPDirClient, groupId uint32) error { + buffer := make(chan *apiv2.StreamExtSnapshotRequest, 10) errCh := make(chan error, 1) ctx := in.Context() + glog.Infof("[import:forward:diffrent-group] started streaming to group [%v]", groupId) go func() { defer close(buffer) for { select { case <-ctx.Done(): - glog.Info("[import] Context cancelled, stopping receive goroutine.") + glog.Info("[import:forward:diffrent-group] Context cancelled, stopping receive goroutine.") errCh <- fmt.Errorf("context deadline exceeded") return default: msg, err := in.Recv() if err != nil { if !errors.Is(err, io.EOF) { - glog.Errorf("[import] Error receiving from in stream: %v", err) + glog.Errorf("[import:forward:diffrent-group] Error receiving from in stream: %v", err) errCh <- err } return @@ -138,58 +186,67 @@ Loop: break Loop } - data := &apiv2.StreamPDirRequest{StreamPacket: &apiv2.StreamPacket{Data: msg.StreamPacket.Data}} + data := &apiv2.StreamExtSnapshotRequest{Pkt: &apiv2.StreamPacket{Data: msg.Pkt.Data}} - if msg.StreamPacket.Done { + if msg.Pkt.Done { d := apiv2.StreamPacket{Done: true} - if err := out.Send(&apiv2.StreamPDirRequest{StreamPacket: &d}); err != nil { - glog.Errorf("Error sending 'done' to out stream: %v", err) - return err + if err := out.Send(&apiv2.StreamExtSnapshotRequest{Pkt: &d}); err != nil { + glog.Errorf(`[import:forward:diffrent-group] Error sending 'done' to out stream for group [%v]: %v`, + groupId, err) + return fmt.Errorf(`[import:forward:diffrent-group] Error sending 'done' to out + stream for group [%v]: %v`, groupId, err) } - glog.Infoln("[import] All key-values have been transferred.") + glog.Infof(`[import:forward:diffrent-group] All key-values have been transferred + to the [%v] group.`, groupId) break Loop } if err := out.Send(data); err != nil { - glog.Errorf("[import] Error sending to outstream: %v", err) - return fmt.Errorf("error sending to outstream: %v", err) + glog.Errorf("[import:forward:diffrent-group] Error sending to outstream for group [%v]: %v", + groupId, err) + return fmt.Errorf("[import:forward:diffrent-group] error sending to outstream for group [%v]: %v", + groupId, err) } - size += len(msg.StreamPacket.Data) - glog.Infof("[import] Sent batch of size: %s. Total so far: %s\n", - humanize.IBytes(uint64(len(msg.StreamPacket.Data))), humanize.IBytes(uint64(size))) + size += len(msg.Pkt.Data) + glog.Infof("[import:forward:diffrent-group] Sent batch of size: %s. Total so far: %s\n", + humanize.IBytes(uint64(len(msg.Pkt.Data))), humanize.IBytes(uint64(size))) } } // Close the incoming stream properly - if err := in.SendAndClose(&apiv2.StreamPDirResponse{Done: true}); err != nil { - return fmt.Errorf("failed to send close on in: %v", err) + if err := in.SendAndClose(&apiv2.StreamExtSnapshotResponse{}); err != nil { + glog.Errorf("[import:forward:diffrent-group] failed to send close on in stream for group [%v]: %v", + groupId, err) + return fmt.Errorf("[import:forward:diffrent-group] failed to send close on in stream for group [%v]: %v", + groupId, err) } // Wait for ACK from the out stream _, err := out.CloseAndRecv() if err != nil { - return fmt.Errorf("failed to receive ACK from out stream: %w", err) + glog.Errorf("[import:forward:diffrent-group] failed to receive ACK from group [%v]: %v", groupId, err) + return fmt.Errorf("[import:forward:diffrent-group] failed to receive ACK from group [%v]: %w", groupId, err) } - glog.Info("[import] Received ACK") + glog.Infof("[import:forward:diffrent-group] Received ACK from group [%v]", groupId) return nil } -// flushKvs receives the stream of data from the client and writes it to BadgerDB. -// It also sends a streams the data to other nodes of the same group and reloads the schema from the DB. -func flushKvs(stream apiv2.Dgraph_StreamPDirServer) error { - if err := processStreamData(stream); err != nil { - return err +func (w *grpcWorker) UpdateExtSnapshotStreamingState(ctx context.Context, + req *apiv2.UpdateExtSnapshotStreamingStateRequest) (*pb.Status, error) { + if req == nil { + return nil, errors.New("UpdateExtSnapshotStreamingStateRequest should not be empty") } - return postStreamProcessing(stream.Context()) -} + if req.Start && req.Finish { + return nil, errors.New("UpdateExtSnapshotStreamingStateRequest should not be both start and finish") + } -func (w *grpcWorker) ApplyDrainmode(ctx context.Context, req *pb.DrainModeRequest) (*pb.Status, error) { - drainMode := &pb.DrainModeRequest{State: req.State} - node := groups().Node - err := node.proposeAndWait(ctx, &pb.Proposal{Drainmode: drainMode}) // Subscribe on given prefixes. + glog.Infof("[import] Applying import mode proposal: %v", req) + extSnapshotStreamingState := &apiv2.UpdateExtSnapshotStreamingStateRequest{Start: req.Start, + Finish: req.Finish, DropData: req.DropData} + err := groups().Node.proposeAndWait(ctx, &pb.Proposal{UpdateExtSnapshotStreamingState: extSnapshotStreamingState}) return &pb.Status{}, err } @@ -198,62 +255,58 @@ func (w *grpcWorker) ApplyDrainmode(ctx context.Context, req *pb.DrainModeReques // It writes the data to BadgerDB, sends an acknowledgment once all data is received, // and proposes to accept the newly added data to other group nodes. func (w *grpcWorker) InternalStreamPDir(stream pb.Worker_InternalStreamPDirServer) error { - if err := processStreamData(stream); err != nil { + glog.Info("[import] updating import mode to false") + defer x.ExtSnapshotStreamingState(false) + // we send forward request to the leader of the group so if this node is leader then forward + // will be true else false + forwardReq, err := stream.Recv() + if err != nil { return err } - // Inform Zero about the new tablets. - return postStreamProcessing(stream.Context()) -} -func processStreamData(stream streamProcessor) error { - sw := pstore.NewStreamWriter() - defer sw.Cancel() - - // Prepare the stream writer, which involves deleting existing data. - if err := sw.Prepare(); err != nil { + if err := streamInGroup(stream, forwardReq.Forward); err != nil { return err } - // Track the total size of key-value data received. - size := 0 - for { - // Receive a batch of key-value pairs from the stream. - req, err := stream.Recv() - if err != nil { - return err - } - - kvs := req.GetStreamPacket() - // Check if all key-value pairs have been received. - if kvs != nil && kvs.Done { - glog.Info("[import] All key-values have been received.") - break - } - - // Increment the total size and log the batch size received. - size += len(kvs.Data) - glog.Infof("[import] Received batch of size: %s. Total so far: %s\n", - humanize.IBytes(uint64(len(kvs.Data))), humanize.IBytes(uint64(size))) + return nil +} - // Write the received data to BadgerDB. - buf := z.NewBufferSlice(kvs.Data) - if err := sw.Write(buf); err != nil { - return err +// RunBadgerStream runs a BadgerDB stream to send key-value pairs to the specified group. +// It creates a new stream at the maximum sequence number and sends the data to the specified group. +// It also sends a final 'done' signal to mark completion. +func RunBadgerStream(ctx context.Context, ps *badger.DB, out apiv2.Dgraph_StreamExtSnapshotClient, groupId uint32) error { + stream := ps.NewStreamAt(math.MaxUint64) + stream.LogPrefix = "Sending P dir to group [" + fmt.Sprintf("%d", groupId) + "]" + stream.KeyToList = nil + stream.Send = func(buf *z.Buffer) error { + p := &apiv2.StreamPacket{Data: buf.Bytes()} + if err := out.Send(&apiv2.StreamExtSnapshotRequest{Pkt: p}); err != nil && !errors.Is(err, io.EOF) { + return fmt.Errorf("failed to send data chunk: %w", err) } + return nil } - // Flush any remaining data to ensure it is written to BadgerDB. - if err := sw.Flush(); err != nil { - return err + // Execute the stream process + if err := stream.Orchestrate(ctx); err != nil { + return fmt.Errorf("stream orchestration failed: %w", err) } - glog.Info("[import] P dir writes DONE. Sending ACK") + // Send the final 'done' signal to mark completion + glog.Infof("Sending completion signal for group [%d]", groupId) + done := &apiv2.StreamPacket{Done: true} - // Send an acknowledgment to the leader indicating completion. - return stream.SendAndClose(&apiv2.StreamPDirResponse{Done: true}) + if err := out.Send(&apiv2.StreamExtSnapshotRequest{Pkt: done}); err != nil && !errors.Is(err, io.EOF) { + return fmt.Errorf("failed to send 'done' signal for group [%d]: %w", groupId, err) + } + + return nil } +// postStreamProcessing handles the post-stream processing of data received from the buffer into the local BadgerDB. +// It loads the schema, updates the membership state, informs zero about tablets, resets caches, applies initial schema, +// applies initial types, and resets the GQL schema store. func postStreamProcessing(ctx context.Context) error { + glog.Info("[import:flush:current-node] post stream processing") if err := schema.LoadFromDb(ctx); err != nil { return errors.Wrapf(err, "cannot load schema after streaming data") } @@ -262,12 +315,298 @@ func postStreamProcessing(ctx context.Context) error { } gr.informZeroAboutTablets() - posting.ResetCache() ResetAclCache() groups().applyInitialSchema() groups().applyInitialTypes() ResetGQLSchemaStore() + glog.Info("[import:flush:current-node] post stream processing done") + + return nil +} + +// streamInGroup handles the streaming of data within a group. +// This function is called on both leader and follower nodes with different behaviors: +// - Leader (forward=true): The leader node receives data and forwards it to all group members +// - Follower (forward=false): The follower node receives data and stores it in local BadgerDB. +// +// Parameters: +// - stream: The gRPC stream server for receiving streaming data +// - forward: Indicates if this node is forwarding data to other nodes +// - true: This node is the group leader and will forward data to group members +// - false: This node is a follower receiving forwarded data and storing locally +// +// The function: +// 1. Creates a context with cancellation support for graceful shutdown +// 2. Sets up a pub/sub system for message distribution +// 3. Uses an error group to manage concurrent operations +// 4. Tracks successful nodes for majority consensus (only relevant for leader) +// 5. Receives messages in a loop until EOF or error +// 6. Publishes received messages to all subscribers (for leader) or stores locally (for follower) +// 7. Handles cleanup and error cases appropriately +// +// Returns: +// - nil: If streaming completes successfully +// - error: If there's an issue receiving data or if majority consensus isn't achieved (for leader) +func streamInGroup(stream apiv2.Dgraph_StreamExtSnapshotServer, forward bool) error { + node := groups().Node + glog.Infof("[import] got stream,forwarding in group [%v]", forward) + ctx := stream.Context() + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + ps := &pubSub{} + eg, ctx := errgroup.WithContext(ctx) + // this we careated for to check the majority + successfulNodes := make(map[string]bool) + + // Receive messages and publish to all subscribers + eg.Go(func() error { + defer ps.close() + for { + select { + case <-ctx.Done(): + glog.Info("[import] Context cancelled, stopping receive goroutine.") + return ctx.Err() + default: + msg, err := stream.Recv() + if err != nil { + if !errors.Is(err, io.EOF) { + glog.Errorf("[import] Error receiving from in stream: %v", err) + return err + } + return nil + } + ps.publish(msg) + } + } + }) + + size := 0 + for _, member := range groups().state.Groups[node.gid].Members { + if member.Addr == node.MyAddr { + buffer := ps.subscribe() + eg.Go(func() error { + if forward { + glog.Infof("[import:flush:current-node] this is a leader node of group [%v]"+ + "flushing P dir to current node [%v]", node.gid, member.Addr) + } + if err := flushInCurrentNode(ctx, buffer, size); err != nil { + return err + } + if forward { + successfulNodes[member.Addr] = true + glog.Infof("[import] Successfully flushed data to node: %v", member.Addr) + } + return nil + }) + continue + } + + // we will not going to return any error from here because we carea about majroity of nodes if the mojority + // of node able to get the data then the behind once can cathup qafter that + if forward { + member := member + buffer := ps.subscribe() + eg.Go(func() error { + glog.Infof(`[import:forward:current-group] streaming P dir to [%v] from [%v]`, member.Addr, node.MyAddr) + if member.AmDead { + glog.Infof(`[import:forward:current-group] [%v] is dead, skipping`, member.Addr) + return nil + } + pl, err := conn.GetPools().Get(member.Addr) + if err != nil { + successfulNodes[member.Addr] = false + glog.Errorf("connection error to [%v]: %v", member.Addr, err) + return nil + } + c := pb.NewWorkerClient(pl.Get()) + peerStream, err := c.InternalStreamPDir(ctx) + if err != nil { + successfulNodes[member.Addr] = false + glog.Errorf("failed to establish stream with peer %v: %v", member.Addr, err) + return nil + } + forwardReq := &apiv2.StreamExtSnapshotRequest{Forward: false} + if err := peerStream.Send(forwardReq); err != nil { + successfulNodes[member.Addr] = false + glog.Errorf("failed to send forward request: %v", err) + return nil + } + + if err := connectToNodeStream(peerStream, buffer, size, member.Addr); err != nil { + successfulNodes[member.Addr] = false + glog.Errorf("failed to connect to node stream: %v", err) + return nil + } + successfulNodes[member.Addr] = true + glog.Infof("[import] Successfully connected and streamed data to node: %v", member.Addr) + return nil + }) + } + } + + if err := eg.Wait(); err != nil { + return err + } + + if err := stream.SendAndClose(&apiv2.StreamExtSnapshotResponse{}); err != nil { + return fmt.Errorf("[import] failed to send close on in: %w", err) + } + + // if we are the leader and we are not able to get the majority of nodes then we will return error + // because we want to make sure that all the nodes are able to get the data + if forward && !checkMajority(successfulNodes) { + glog.Error("[import] Majority of nodes failed to receive data.") + return errors.New("Failed to achieve majority consensus") + } return nil } + +// Calculate majority based on Raft quorum rules with special handling for small clusters +func checkMajority(successfulNodes map[string]bool) bool { + totalNodes := len(successfulNodes) + successfulCount := 0 + + for _, success := range successfulNodes { + if success { + successfulCount++ + } + } + + // Special cases for small clusters + switch totalNodes { + case 0: + // No nodes - this should never happen + glog.Error("[import] No nodes in cluster") + return false + case 1: + // Single node - must succeed + return successfulCount == 1 + case 2: + // Two nodes - both must succeed + return successfulCount == 2 + default: + // Regular Raft quorum rule for 3+ nodes + majority := totalNodes/2 + 1 + return successfulCount >= majority + } +} + +// connectToNodeStream handles the connection to a peer node for streaming data. +// It receives data from the buffer and sends it to the peer node using the provided stream. +// +// Parameters: +// - out: The gRPC stream client for sending data to the peer node +// - buffer: A channel of StreamPDirRequest messages containing the data to be sent +// - size: The size of the data being sent (updated in-place) +// - peerId: The identifier of the peer node +// +// The function: +// 1. Receives messages from the buffer in a loop until EOF or error +// 2. Sends each message to the peer node using the stream +// 3. Updates the size of the data being sent +// 4. Sends a 'done' signal to the peer node when all data has been sent +// 5. Handles cleanup and error cases appropriately +// +// Returns: +// - nil: If the connection is successful and all data is sent +// - error: If there's an issue sending data or if the context deadline is exceeded +func connectToNodeStream(out pb.Worker_InternalStreamPDirClient, buffer <-chan *apiv2.StreamExtSnapshotRequest, + size int, peerId string) error { + ctx := out.Context() +Loop: + for { + select { + case <-ctx.Done(): + return fmt.Errorf("context deadline exceeded") + + default: + msg, ok := <-buffer + if !ok { + break Loop + } + + data := &apiv2.StreamExtSnapshotRequest{Pkt: &apiv2.StreamPacket{Data: msg.Pkt.Data}} + + if msg.Pkt.Done { + glog.Infof("[import:forward:current-group] received done signal from [%v]", peerId) + d := apiv2.StreamPacket{Done: true} + if err := out.Send(&apiv2.StreamExtSnapshotRequest{Pkt: &d}); err != nil { + return err + } + break Loop + } + + if err := out.Send(data); err != nil { + return err + } + + size += len(msg.Pkt.Data) + } + } + + _, err := out.CloseAndRecv() + if err != nil { + return fmt.Errorf("[import:forward:current-group] failed to receive ACK from [%v]: %w", peerId, err) + } + + glog.Infof("[import:forward:current-group] successfully streamed to [%v]", peerId) + return nil +} + +// flushInCurrentNode handles the flushing of data received from the buffer into the local BadgerDB. +// +// Parameters: +// - ctx: The context for the operation +// - buffer: A channel of StreamPDirRequest messages containing the data to be flushed +// - size: The size of the data being flushed (updated in-place) +// +// The function: +// 1. Receives messages from the buffer in a loop until EOF or error +// 2. Flushed each message to the local BadgerDB using a stream writer +// 3. Updates the size of the data being flushed +// 4. Sends a 'done' signal to the peer node when all data has been sent +// 5. Handles cleanup and error cases appropriately +// +// Returns: +// - nil: If the flushing is successful and all data is sent +// - error: If there's an issue flushing data or if the context deadline is exceeded +func flushInCurrentNode(ctx context.Context, buffer <-chan *apiv2.StreamExtSnapshotRequest, size int) error { + glog.Infof("[import:flush:current-node] flushing P dir in badger db") + sw := pstore.NewStreamWriter() + defer sw.Cancel() + if err := sw.Prepare(); err != nil { + return err + } +Loop: + for { + select { + case <-ctx.Done(): + return fmt.Errorf("context deadline exceeded") + + default: + msg, ok := <-buffer + if !ok { + break Loop + } + kvs := msg.GetPkt() + if kvs != nil && kvs.Done { + break + } + + size += len(kvs.Data) + buf := z.NewBufferSlice(kvs.Data) + if err := sw.Write(buf); err != nil { + return err + } + } + } + + if err := sw.Flush(); err != nil { + return err + } + glog.Infof("[import:flush:current-node] successfully flushed data in badger db") + return postStreamProcessing(ctx) +} diff --git a/x/health.go b/x/health.go index 567986b52b1..3e21c1fc39c 100644 --- a/x/health.go +++ b/x/health.go @@ -8,6 +8,7 @@ package x import ( "sync/atomic" + "github.com/golang/glog" "github.com/pkg/errors" ) @@ -15,10 +16,12 @@ var ( // the drainingMode variable should be accessed through the atomic.Store and atomic.Load // functions. The value 0 means the draining-mode is disabled, and the value 1 means the // mode is enabled - drainingMode uint32 + drainingMode uint32 + extSnapshotStreamingState uint32 healthCheck uint32 errHealth = errors.New("Please retry again, server is not ready to accept requests") + errImportMode = errors.New("Please retry again, server is not ready to accept requests") errDrainingMode = errors.New("the server is in draining mode " + "and client requests will only be allowed after exiting the mode " + " by sending a GraphQL draining(enable: false) mutation to /admin") @@ -34,6 +37,18 @@ func UpdateDrainingMode(enable bool) { setStatus(&drainingMode, enable) } +// ExtSnapshotStreamingState updates the server's import mode +func ExtSnapshotStreamingState(enable bool) { + glog.Info("[import] Updating import mode to ", enable) + setStatus(&extSnapshotStreamingState, enable) + setStatus(&drainingMode, enable) +} + +// IsExtSnapshotStreamingStateTrue returns whether the server is in import mode or not +func IsExtSnapshotStreamingStateTrue() bool { + return atomic.LoadUint32(&extSnapshotStreamingState) == 1 +} + // HealthCheck returns whether the server is ready to accept requests or not // Load balancer would add the node to the endpoint once health check starts // returning true From b1b96ead526fb8d85d88ad11414bbdf3894925dc Mon Sep 17 00:00:00 2001 From: shivaji-dgraph Date: Thu, 29 May 2025 15:13:26 +0530 Subject: [PATCH 2/4] fix trunk --- x/health.go | 1 - 1 file changed, 1 deletion(-) diff --git a/x/health.go b/x/health.go index 3e21c1fc39c..0d3b3121a85 100644 --- a/x/health.go +++ b/x/health.go @@ -21,7 +21,6 @@ var ( healthCheck uint32 errHealth = errors.New("Please retry again, server is not ready to accept requests") - errImportMode = errors.New("Please retry again, server is not ready to accept requests") errDrainingMode = errors.New("the server is in draining mode " + "and client requests will only be allowed after exiting the mode " + " by sending a GraphQL draining(enable: false) mutation to /admin") From 1f2eaad299a084c158c2d8b480893c90259e3619 Mon Sep 17 00:00:00 2001 From: shivaji-dgraph Date: Thu, 29 May 2025 16:42:57 +0530 Subject: [PATCH 3/4] fix tests --- dgraph/cmd/dgraphimport/import_client.go | 9 +++++++++ worker/import.go | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/dgraph/cmd/dgraphimport/import_client.go b/dgraph/cmd/dgraphimport/import_client.go index 5e6a56a7dad..9bd41c964d2 100644 --- a/dgraph/cmd/dgraphimport/import_client.go +++ b/dgraph/cmd/dgraphimport/import_client.go @@ -100,6 +100,15 @@ func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups } glog.Info("Completed streaming all pdirs") + req := &apiv2.UpdateExtSnapshotStreamingStateRequest{ + Start: false, + Finish: true, + DropData: false, + } + if _, err := dc.UpdateExtSnapshotStreamingState(context.Background(), req); err != nil { + return fmt.Errorf("failed to disabled drain mode: %v", err) + } + glog.Infof("Completed streaming all pdirs") return nil } diff --git a/worker/import.go b/worker/import.go index 488256a40de..89d85478d35 100644 --- a/worker/import.go +++ b/worker/import.go @@ -357,7 +357,7 @@ func streamInGroup(stream apiv2.Dgraph_StreamExtSnapshotServer, forward bool) er ps := &pubSub{} eg, ctx := errgroup.WithContext(ctx) - // this we careated for to check the majority + // We created this to check the majority successfulNodes := make(map[string]bool) // Receive messages and publish to all subscribers From 687d52172bd6e803efdae0e8951823f73d4ce636 Mon Sep 17 00:00:00 2001 From: shivaji-dgraph Date: Thu, 29 May 2025 18:15:24 +0530 Subject: [PATCH 4/4] resolve review comments --- dgraph/cmd/dgraphimport/import_client.go | 34 ++++++++-------- dgraph/cmd/dgraphimport/import_test.go | 51 ++++++++++++------------ worker/draft.go | 3 +- worker/import.go | 4 +- 4 files changed, 44 insertions(+), 48 deletions(-) diff --git a/dgraph/cmd/dgraphimport/import_client.go b/dgraph/cmd/dgraphimport/import_client.go index 9bd41c964d2..aed27cbc8b9 100644 --- a/dgraph/cmd/dgraphimport/import_client.go +++ b/dgraph/cmd/dgraphimport/import_client.go @@ -27,7 +27,7 @@ func newClient(endpoint string, opts grpc.DialOption) (apiv2.DgraphClient, error return nil, fmt.Errorf("failed to connect to endpoint [%s]: %w", endpoint, err) } - glog.Infof("Successfully connected to Dgraph endpoint: %s", endpoint) + glog.Infof("[import:ext-snapshot] Successfully connected to Dgraph endpoint: %s", endpoint) return apiv2.NewDgraphClient(conn), nil } @@ -46,16 +46,16 @@ func Import(ctx context.Context, endpoint string, opts grpc.DialOption, bulkOutD // startPDirStream initiates a snapshot stream session with the Dgraph server. func startPDirStream(ctx context.Context, dc apiv2.DgraphClient) (*apiv2.UpdateExtSnapshotStreamingStateResponse, error) { - glog.Info("Initiating pdir stream") + glog.Info("[import:ext-snapshot] Initiating pdir stream") req := &apiv2.UpdateExtSnapshotStreamingStateRequest{ Start: true, } resp, err := dc.UpdateExtSnapshotStreamingState(ctx, req) if err != nil { - glog.Errorf("failed to initiate pdir stream: %v", err) + glog.Errorf("[import:ext-snapshot] failed to initiate pdir stream: %v", err) return nil, fmt.Errorf("failed to initiate pdir stream: %v", err) } - glog.Info("Pdir stream initiated successfully") + glog.Info("[import:ext-snapshot] Pdir stream initiated successfully") return resp, nil } @@ -63,7 +63,7 @@ func startPDirStream(ctx context.Context, dc apiv2.DgraphClient) (*apiv2.UpdateE // p directory to the corresponding group IDs. It first scans the provided directory for // subdirectories named with numeric group IDs. func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups []uint32) error { - glog.Infof("Starting to stream pdir from directory: %s", baseDir) + glog.Infof("[import:ext-snapshot] Starting to stream pdir from directory: %s", baseDir) errG, ctx := errgroup.WithContext(ctx) for _, group := range groups { @@ -73,9 +73,9 @@ func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups if _, err := os.Stat(pDir); err != nil { return fmt.Errorf("p directory does not exist for group [%d]: [%s]", group, pDir) } - glog.Infof("Streaming data for group [%d] from directory: [%s]", group, pDir) + glog.Infof("[import:ext-snapshot] Streaming data for group [%d] from directory: [%s]", group, pDir) if err := streamData(ctx, dc, pDir, group); err != nil { - glog.Errorf("Failed to stream data for groups [%v] from directory: [%s]: %v", group, pDir, err) + glog.Errorf("[import:ext-snapshot] Failed to stream data for groups [%v] from directory: [%s]: %v", group, pDir, err) return err } @@ -83,6 +83,7 @@ func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups }) } if err1 := errG.Wait(); err1 != nil { + glog.Errorf("[import:ext-snapshot] something went wrong while streaming p directory: %v", err1) // If the p directory doesn't exist for this group, it indicates that // streaming might be in progress to other groups. We disable drain mode // to prevent interference and drop any streamed data to ensure a clean state. @@ -92,14 +93,14 @@ func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups DropData: true, } if _, err := dc.UpdateExtSnapshotStreamingState(context.Background(), req); err != nil { - return fmt.Errorf("failed to stream data :%v failed to off drain mode: %v", err1, err) + return fmt.Errorf("failed to turn off drain mode: %v", err) } - glog.Info("successfully disabled drain mode") + glog.Info("[import:ext-snapshot]successfully disabled drain mode") return err1 } - glog.Info("Completed streaming all pdirs") + glog.Info("[import:ext-snapshot] Completed streaming all pdirs") req := &apiv2.UpdateExtSnapshotStreamingStateRequest{ Start: false, Finish: true, @@ -108,7 +109,7 @@ func sendPDir(ctx context.Context, dc apiv2.DgraphClient, baseDir string, groups if _, err := dc.UpdateExtSnapshotStreamingState(context.Background(), req); err != nil { return fmt.Errorf("failed to disabled drain mode: %v", err) } - glog.Infof("Completed streaming all pdirs") + glog.Info("[import:ext-snapshot] successfully disabled drain mode") return nil } @@ -127,27 +128,24 @@ func streamData(ctx context.Context, dc apiv2.DgraphClient, pdir string, groupId opt := badger.DefaultOptions(pdir) ps, err := badger.OpenManaged(opt) if err != nil { - return fmt.Errorf("failed to open BadgerDB at [%s]: %w", pdir, err) + return fmt.Errorf("[import:ext-snapshot] failed to open BadgerDB at [%s]: %w", pdir, err) } defer func() { if err := ps.Close(); err != nil { - glog.Warningf("Error closing BadgerDB: %v", err) + glog.Warningf("[import:ext-snapshot] Error closing BadgerDB: %v", err) } }() // Send group ID as the first message in the stream - glog.Infof("Sending group ID [%d] to server", groupId) + glog.Infof("[import:ext-snapshot] Sending group ID [%d] to server", groupId) groupReq := &apiv2.StreamExtSnapshotRequest{GroupId: groupId} if err := out.Send(groupReq); err != nil { return fmt.Errorf("failed to send group ID [%d]: %w", groupId, err) } // Configure and start the BadgerDB stream - glog.Infof("Starting BadgerDB stream for group [%d]", groupId) - // if err := RunBadgerStream(ctx, ps, out, groupId); err != nil { - // return fmt.Errorf("badger stream failed for group [%d]: %w", groupId, err) - // } + glog.Infof("[import:ext-snapshot] Starting BadgerDB stream for group [%d]", groupId) if err := worker.RunBadgerStream(ctx, ps, out, groupId); err != nil { return fmt.Errorf("badger stream failed for group [%d]: %w", groupId, err) diff --git a/dgraph/cmd/dgraphimport/import_test.go b/dgraph/cmd/dgraphimport/import_test.go index d86443dc17c..42d17f295cf 100644 --- a/dgraph/cmd/dgraphimport/import_test.go +++ b/dgraph/cmd/dgraphimport/import_test.go @@ -27,6 +27,18 @@ import ( "google.golang.org/protobuf/encoding/protojson" ) +type tc struct { + name string + bulkAlphas int + targetAlphas int + replicasFactor int + downAlphas int + negativeTestCase bool + description string + err string + waitForSnapshot bool +} + const expectedSchema = `{ "schema": [ {"predicate":"actor.film","type":"uid","count":true,"list":true}, @@ -96,17 +108,7 @@ func TestDrainModeAfterStartSnapshotStream(t *testing.T) { // TestImportApis tests import functionality with different cluster configurations func TestImportApis(t *testing.T) { - tests := []struct { - name string - bulkAlphas int // Number of alphas in source cluster - targetAlphas int // Number of alphas in target cluster - replicasFactor int // Number of replicas for each group - downAlphas int // Number of alphas to be shutdown - negativeTestCase bool // True if this is an expected failure case - description string - err string - waitForSnapshot bool - }{ + tests := []tc{ { name: "SingleGroupShutTwoAlphasPerGroup", bulkAlphas: 1, @@ -219,18 +221,16 @@ func TestImportApis(t *testing.T) { } else { t.Logf("Running test case: %s", tt.description) } - runImportTest(t, tt.bulkAlphas, tt.targetAlphas, tt.replicasFactor, tt.downAlphas, tt.negativeTestCase, - tt.err, tt.waitForSnapshot) + runImportTest(t, tt) }) } } -func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDownAlphas int, negative bool, - errStr string, waitForSnapshot bool) { - bulkCluster, baseDir := setupBulkCluster(t, bulkAlphas) +func runImportTest(t *testing.T, tt tc) { + bulkCluster, baseDir := setupBulkCluster(t, tt.bulkAlphas) defer func() { bulkCluster.Cleanup(t.Failed()) }() - targetCluster, gc, gcCleanup := setupTargetCluster(t, targetAlphas, replicasFactor) + targetCluster, gc, gcCleanup := setupTargetCluster(t, tt.targetAlphas, tt.replicasFactor) defer func() { targetCluster.Cleanup(t.Failed()) }() defer gcCleanup() @@ -254,8 +254,6 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDo healthResp, err := hc.GetAlphaState() require.NoError(t, err) require.NoError(t, protojson.Unmarshal(healthResp, &state)) - fmt.Println("Health response: ", string(healthResp)) - // Group alphas by their group number alphaGroups := make(map[uint32][]int) for _, group := range state.Groups { @@ -273,18 +271,18 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDo // Shutdown specified number of alphas from each group for group, alphas := range alphaGroups { - for i := 0; i < numDownAlphas; i++ { + for i := 0; i < tt.downAlphas; i++ { alphaID := alphas[i] t.Logf("Shutting down alpha %v from group %v", alphaID, group) require.NoError(t, targetCluster.StopAlpha(alphaID)) } } - if negative { + if tt.negativeTestCase { err := Import(context.Background(), url, grpc.WithTransportCredentials(insecure.NewCredentials()), outDir) require.Error(t, err) fmt.Println("Error: ", err) - require.ErrorContains(t, err, errStr) + require.ErrorContains(t, err, tt.err) return } @@ -292,7 +290,7 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDo grpc.WithTransportCredentials(insecure.NewCredentials()), outDir)) for group, alphas := range alphaGroups { - for i := 0; i < numDownAlphas; i++ { + for i := 0; i < tt.downAlphas; i++ { alphaID := alphas[i] t.Logf("Starting alpha %v from group %v", alphaID, group) require.NoError(t, targetCluster.StartAlpha(alphaID)) @@ -301,9 +299,9 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDo require.NoError(t, targetCluster.HealthCheck(false)) - if waitForSnapshot { + if tt.waitForSnapshot { for grp, alphas := range alphaGroups { - for i := 0; i < numDownAlphas; i++ { + for i := 0; i < tt.downAlphas; i++ { fmt.Println("Waiting for snapshot for alpha", alphas[i], "group", grp) hc, err := targetCluster.GetAlphaHttpClient(alphas[i]) require.NoError(t, err) @@ -318,7 +316,8 @@ func runImportTest(t *testing.T, bulkAlphas, targetAlphas, replicasFactor, numDo t.Log("Import completed") - for i := 0; i < targetAlphas; i++ { + for i := 0; i < tt.targetAlphas; i++ { + t.Logf("Verifying import for alpha %v", i) gc, cleanup, err := targetCluster.AlphaClient(i) require.NoError(t, err) defer cleanup() diff --git a/worker/draft.go b/worker/draft.go index ae2f17d396a..bf5baf68eff 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -719,8 +719,7 @@ func (n *node) applyCommitted(proposal *pb.Proposal, key uint64) error { // was not there). We can't be certain if we have disabled x.UpdateExtSnapshotStreamingState or // not, so we are disabling it here: x.ExtSnapshotStreamingState(false) - err := posting.DeleteAll() - if err != nil { + if err := posting.DeleteAll(); err != nil { glog.Errorf("[import] failed to delete all data: %v", err) return err } diff --git a/worker/import.go b/worker/import.go index 89d85478d35..5a81eea9fc3 100644 --- a/worker/import.go +++ b/worker/import.go @@ -276,7 +276,7 @@ func (w *grpcWorker) InternalStreamPDir(stream pb.Worker_InternalStreamPDirServe // It also sends a final 'done' signal to mark completion. func RunBadgerStream(ctx context.Context, ps *badger.DB, out apiv2.Dgraph_StreamExtSnapshotClient, groupId uint32) error { stream := ps.NewStreamAt(math.MaxUint64) - stream.LogPrefix = "Sending P dir to group [" + fmt.Sprintf("%d", groupId) + "]" + stream.LogPrefix = "[import:ext-snapshot] Sending P dir to group [" + fmt.Sprintf("%d", groupId) + "]" stream.KeyToList = nil stream.Send = func(buf *z.Buffer) error { p := &apiv2.StreamPacket{Data: buf.Bytes()} @@ -292,7 +292,7 @@ func RunBadgerStream(ctx context.Context, ps *badger.DB, out apiv2.Dgraph_Stream } // Send the final 'done' signal to mark completion - glog.Infof("Sending completion signal for group [%d]", groupId) + glog.Infof("[import:ext-snapshot] Sending completion signal for group [%d]", groupId) done := &apiv2.StreamPacket{Done: true} if err := out.Send(&apiv2.StreamExtSnapshotRequest{Pkt: done}); err != nil && !errors.Is(err, io.EOF) {