From ad87da565854ec8c3bf344ceed5fc91fc31e2a1b Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 12 Oct 2023 15:09:20 -0400 Subject: [PATCH 1/2] Add gRPC method to get the status of a mirror --- docker-compose.yml | 8 +- flow/activities/flowable.go | 8 +- flow/cmd/handler.go | 56 +- flow/cmd/mirror_status.go | 256 ++++ flow/generated/protos/route.pb.go | 805 +++++++++++-- flow/generated/protos/route_grpc.pb.go | 37 + flow/workflows/snapshot_flow.go | 3 + nexus/analyzer/src/lib.rs | 14 +- .../migrations/V7__store_flow_config.sql | 2 + nexus/pt/src/peerdb_route.rs | 79 ++ nexus/pt/src/peerdb_route.serde.rs | 1045 +++++++++++++++-- nexus/pt/src/peerdb_route.tonic.rs | 80 ++ protos/route.proto | 46 + ui/grpc_generated/route.ts | 743 ++++++++++++ 14 files changed, 3026 insertions(+), 156 deletions(-) create mode 100644 flow/cmd/mirror_status.go create mode 100644 nexus/catalog/migrations/V7__store_flow_config.sql diff --git a/docker-compose.yml b/docker-compose.yml index 6da2cadfb2..ec6ad41ed6 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -10,7 +10,7 @@ x-catalog-config: &catalog-config x-flow-worker-env: &flow-worker-env TEMPORAL_HOST_PORT: temporal:7233 # For GCS, these will be your HMAC keys instead - # For more information: + # For more information: # https://cloud.google.com/storage/docs/authentication/managing-hmackeys AWS_ACCESS_KEY_ID: ${AWS_ACCESS_KEY_ID:-} AWS_SECRET_ACCESS_KEY: ${AWS_SECRET_ACCESS_KEY:-} @@ -59,7 +59,7 @@ services: - POSTGRES_PWD=postgres - POSTGRES_SEEDS=catalog - DYNAMIC_CONFIG_FILE_PATH=config/dynamicconfig/development-sql.yaml - image: temporalio/auto-setup:1.21.3 + image: temporalio/auto-setup:1.22.2 ports: - 7233:7233 volumes: @@ -80,7 +80,7 @@ services: environment: - TEMPORAL_ADDRESS=temporal:7233 - TEMPORAL_CLI_ADDRESS=temporal:7233 - image: temporalio/admin-tools:1.21.3 + image: temporalio/admin-tools:1.22.2 stdin_open: true tty: true healthcheck: @@ -208,7 +208,6 @@ services: - multi-metrics - metrics - peerdb-grafana: container_name: peerdb-grafana build: @@ -223,7 +222,6 @@ services: - multi-metrics - metrics - peerdb-ui: container_name: peerdb-ui build: diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 0eb93920cd..5e639a2963 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -424,8 +424,12 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, return nil, fmt.Errorf("failed to get partitions from source: %w", err) } if len(partitions) > 0 { - err = a.CatalogMirrorMonitor.InitializeQRepRun(ctx, config.FlowJobName, - runUUID, startTime) + err = a.CatalogMirrorMonitor.InitializeQRepRun( + ctx, + config.FlowJobName, + runUUID, + startTime, + ) if err != nil { return nil, err } diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index e8c21531b6..1252fb844a 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -117,8 +117,14 @@ func (h *FlowRequestHandler) CreateCDCFlow( } } + var err error + err = h.updateFlowConfigInCatalog(cfg) + if err != nil { + return nil, fmt.Errorf("unable to update flow config in catalog: %w", err) + } + state := peerflow.NewCDCFlowState() - _, err := h.temporalClient.ExecuteWorkflow( + _, err = h.temporalClient.ExecuteWorkflow( ctx, // context workflowOptions, // workflow start options peerflow.CDCFlowWorkflowWithConfig, // workflow function @@ -135,6 +141,27 @@ func (h *FlowRequestHandler) CreateCDCFlow( }, nil } +func (h *FlowRequestHandler) updateFlowConfigInCatalog( + cfg *protos.FlowConnectionConfigs, +) error { + var cfgBytes []byte + var err error + + cfgBytes, err = proto.Marshal(cfg) + if err != nil { + return fmt.Errorf("unable to marshal flow config: %w", err) + } + + _, err = h.pool.Exec(context.Background(), + "UPDATE flows SET config_proto = $1 WHERE name = $2", + cfgBytes, cfg.FlowJobName) + if err != nil { + return fmt.Errorf("unable to update flow config in catalog: %w", err) + } + + return nil +} + func (h *FlowRequestHandler) CreateQRepFlow( ctx context.Context, req *protos.CreateQRepFlowRequest) (*protos.CreateQRepFlowResponse, error) { lastPartition := &protos.QRepPartition{ @@ -162,11 +189,38 @@ func (h *FlowRequestHandler) CreateQRepFlow( return nil, fmt.Errorf("unable to start QRepFlow workflow: %w", err) } + err = h.updateQRepConfigInCatalog(cfg) + if err != nil { + return nil, fmt.Errorf("unable to update qrep config in catalog: %w", err) + } + return &protos.CreateQRepFlowResponse{ WorflowId: workflowID, }, nil } +// updateQRepConfigInCatalog updates the qrep config in the catalog +func (h *FlowRequestHandler) updateQRepConfigInCatalog( + cfg *protos.QRepConfig, +) error { + var cfgBytes []byte + var err error + + cfgBytes, err = proto.Marshal(cfg) + if err != nil { + return fmt.Errorf("unable to marshal qrep config: %w", err) + } + + _, err = h.pool.Exec(context.Background(), + "UPDATE flows SET config_proto = $1 WHERE name = $2", + cfgBytes, cfg.FlowJobName) + if err != nil { + return fmt.Errorf("unable to update qrep config in catalog: %w", err) + } + + return nil +} + func (h *FlowRequestHandler) ShutdownFlow( ctx context.Context, req *protos.ShutdownRequest, diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go new file mode 100644 index 0000000000..19d1d506d3 --- /dev/null +++ b/flow/cmd/mirror_status.go @@ -0,0 +1,256 @@ +package main + +import ( + "context" + "database/sql" + "fmt" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgtype" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" +) + +func (h *FlowRequestHandler) MirrorStatus( + ctx context.Context, + req *protos.MirrorStatusRequest, +) (*protos.MirrorStatusResponse, error) { + cdcFlow, err := h.isCDCFlow(ctx, req.FlowJobName) + if err != nil { + return &protos.MirrorStatusResponse{ + ErrorMessage: fmt.Sprintf("unable to query flow: %s", err.Error()), + }, nil + } + + if cdcFlow { + cdcStatus, err := h.CDCFlowStatus(ctx, req) + if err != nil { + return &protos.MirrorStatusResponse{ + ErrorMessage: fmt.Sprintf("unable to query flow: %s", err.Error()), + }, nil + } + + return &protos.MirrorStatusResponse{ + FlowJobName: req.FlowJobName, + Status: &protos.MirrorStatusResponse_CdcStatus{ + CdcStatus: cdcStatus, + }, + }, nil + } else { + qrepStatus, err := h.QRepFlowStatus(ctx, req) + if err != nil { + return &protos.MirrorStatusResponse{ + ErrorMessage: fmt.Sprintf("unable to query flow: %s", err.Error()), + }, nil + } + + return &protos.MirrorStatusResponse{ + FlowJobName: req.FlowJobName, + Status: &protos.MirrorStatusResponse_QrepStatus{ + QrepStatus: qrepStatus, + }, + }, nil + } +} + +func (h *FlowRequestHandler) CDCFlowStatus( + ctx context.Context, + req *protos.MirrorStatusRequest, +) (*protos.CDCMirrorStatus, error) { + config, err := h.getFlowConfigFromCatalog(req.FlowJobName) + if err != nil { + return nil, err + } + + var initialCopyStatus *protos.SnapshotStatus + if config.DoInitialCopy { + cloneJobNames, err := h.getCloneTableFlowNames(ctx, req.FlowJobName) + if err != nil { + return nil, err + } + + cloneStatuses := []*protos.QRepMirrorStatus{} + for _, cloneJobName := range cloneJobNames { + cloneStatus, err := h.QRepFlowStatus(ctx, &protos.MirrorStatusRequest{ + FlowJobName: cloneJobName, + }) + if err != nil { + return nil, err + } + cloneStatuses = append(cloneStatuses, cloneStatus) + } + + initialCopyStatus = &protos.SnapshotStatus{ + Clones: cloneStatuses, + } + } + + return &protos.CDCMirrorStatus{ + Config: config, + SnapshotStatus: initialCopyStatus, + }, nil +} + +func (h *FlowRequestHandler) QRepFlowStatus( + ctx context.Context, + req *protos.MirrorStatusRequest, +) (*protos.QRepMirrorStatus, error) { + parts, err := h.getPartitionUUIDs(ctx, req.FlowJobName) + if err != nil { + return nil, err + } + + partitionStatuses := []*protos.PartitionStatus{} + for _, part := range parts { + partitionStatus, err := h.getPartitionStatus(ctx, part) + if err != nil { + return nil, err + } + + partitionStatuses = append(partitionStatuses, partitionStatus) + } + + return &protos.QRepMirrorStatus{ + // The clone table jobs that are children of the CDC snapshot flow + // do not have a config entry, so allow this to be nil. + Config: h.getQRepConfigFromCatalog(req.FlowJobName), + Partitions: partitionStatuses, + }, nil +} + +// getPartitionStatus returns the status of a partition uuid. +func (h *FlowRequestHandler) getPartitionStatus( + ctx context.Context, + partitionUUID string, +) (*protos.PartitionStatus, error) { + partitionStatus := &protos.PartitionStatus{ + PartitionId: partitionUUID, + } + + var startTime pgtype.Timestamp + var endTime pgtype.Timestamp + var numRows pgtype.Int4 + + q := "SELECT start_time, end_time, rows_in_partition FROM peerdb_stats.qrep_partitions WHERE partition_uuid = $1" + err := h.pool.QueryRow(ctx, q, partitionUUID).Scan(&startTime, &endTime, &numRows) + if err != nil { + return nil, fmt.Errorf("unable to query qrep partition - %s: %w", partitionUUID, err) + } + + if startTime.Valid { + partitionStatus.StartTime = timestamppb.New(startTime.Time) + } + + if endTime.Valid { + partitionStatus.EndTime = timestamppb.New(endTime.Time) + } + + if numRows.Valid { + partitionStatus.NumRows = numRows.Int32 + } + + return partitionStatus, nil +} + +func (h *FlowRequestHandler) getPartitionUUIDs( + ctx context.Context, + flowJobName string, +) ([]string, error) { + rows, err := h.pool.Query(ctx, + "SELECT partition_uuid FROM peerdb_stats.qrep_partitions WHERE flow_name = $1", flowJobName) + if err != nil { + return nil, fmt.Errorf("unable to query qrep partitions: %w", err) + } + defer rows.Close() + + partitionUUIDs := []string{} + for rows.Next() { + var partitionUUID pgtype.Text + if err := rows.Scan(&partitionUUID); err != nil { + return nil, fmt.Errorf("unable to scan partition row: %w", err) + } + partitionUUIDs = append(partitionUUIDs, partitionUUID.String) + } + + return partitionUUIDs, nil +} + +func (h *FlowRequestHandler) getFlowConfigFromCatalog( + flowJobName string, +) (*protos.FlowConnectionConfigs, error) { + var configBytes sql.RawBytes + var err error + var config protos.FlowConnectionConfigs + + err = h.pool.QueryRow(context.Background(), + "SELECT config_proto FROM flows WHERE name = $1", flowJobName).Scan(&configBytes) + if err != nil { + return nil, fmt.Errorf("unable to query flow config from catalog: %w", err) + } + + err = proto.Unmarshal(configBytes, &config) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal flow config: %w", err) + } + + return &config, nil +} + +func (h *FlowRequestHandler) getQRepConfigFromCatalog( + flowJobName string, +) *protos.QRepConfig { + var configBytes []byte + var err error + var config protos.QRepConfig + + err = h.pool.QueryRow(context.Background(), + "SELECT config_proto FROM flows WHERE name = $1", flowJobName).Scan(&configBytes) + if err != nil { + logrus.Warnf("unable to query qrep config from catalog: %s", err.Error()) + return nil + } + + err = proto.Unmarshal(configBytes, &config) + if err != nil { + return nil + } + + return &config +} + +func (h *FlowRequestHandler) isCDCFlow(ctx context.Context, flowJobName string) (bool, error) { + var query pgtype.Text + err := h.pool.QueryRow(ctx, "SELECT query_string FROM flows WHERE name = $1", flowJobName).Scan(&query) + if err != nil { + return false, fmt.Errorf("unable to query flow: %w", err) + } + + if !query.Valid || len(query.String) == 0 { + return true, nil + } + + return false, nil +} + +func (h *FlowRequestHandler) getCloneTableFlowNames(ctx context.Context, flowJobName string) ([]string, error) { + q := "SELECT flow_name FROM peerdb_stats.qrep_runs WHERE flow_name ILIKE $1" + rows, err := h.pool.Query(ctx, q, "clone_"+flowJobName+"_%") + if err != nil { + return nil, fmt.Errorf("unable to getCloneTableFlowNames: %w", err) + } + defer rows.Close() + + flowNames := []string{} + for rows.Next() { + var name pgtype.Text + if err := rows.Scan(&name); err != nil { + return nil, fmt.Errorf("unable to scan flow row: %w", err) + } + if name.Valid { + flowNames = append(flowNames, name.String) + } + } + + return flowNames, nil +} diff --git a/flow/generated/protos/route.pb.go b/flow/generated/protos/route.pb.go index 0918c82794..e2055c9da6 100644 --- a/flow/generated/protos/route.pb.go +++ b/flow/generated/protos/route.pb.go @@ -9,7 +9,7 @@ package protos import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" - _ "google.golang.org/protobuf/types/known/timestamppb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" ) @@ -730,6 +730,465 @@ func (x *CreatePeerResponse) GetMessage() string { return "" } +type MirrorStatusRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FlowJobName string `protobuf:"bytes,1,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` +} + +func (x *MirrorStatusRequest) Reset() { + *x = MirrorStatusRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MirrorStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MirrorStatusRequest) ProtoMessage() {} + +func (x *MirrorStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[12] + 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 MirrorStatusRequest.ProtoReflect.Descriptor instead. +func (*MirrorStatusRequest) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{12} +} + +func (x *MirrorStatusRequest) GetFlowJobName() string { + if x != nil { + return x.FlowJobName + } + return "" +} + +type PartitionStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PartitionId string `protobuf:"bytes,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + StartTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + NumRows int32 `protobuf:"varint,4,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` +} + +func (x *PartitionStatus) Reset() { + *x = PartitionStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionStatus) ProtoMessage() {} + +func (x *PartitionStatus) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[13] + 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 PartitionStatus.ProtoReflect.Descriptor instead. +func (*PartitionStatus) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{13} +} + +func (x *PartitionStatus) GetPartitionId() string { + if x != nil { + return x.PartitionId + } + return "" +} + +func (x *PartitionStatus) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *PartitionStatus) GetEndTime() *timestamppb.Timestamp { + if x != nil { + return x.EndTime + } + return nil +} + +func (x *PartitionStatus) GetNumRows() int32 { + if x != nil { + return x.NumRows + } + return 0 +} + +type QRepMirrorStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Config *QRepConfig `protobuf:"bytes,1,opt,name=config,proto3" json:"config,omitempty"` + Partitions []*PartitionStatus `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"` +} + +func (x *QRepMirrorStatus) Reset() { + *x = QRepMirrorStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QRepMirrorStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QRepMirrorStatus) ProtoMessage() {} + +func (x *QRepMirrorStatus) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[14] + 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 QRepMirrorStatus.ProtoReflect.Descriptor instead. +func (*QRepMirrorStatus) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{14} +} + +func (x *QRepMirrorStatus) GetConfig() *QRepConfig { + if x != nil { + return x.Config + } + return nil +} + +func (x *QRepMirrorStatus) GetPartitions() []*PartitionStatus { + if x != nil { + return x.Partitions + } + return nil +} + +type CDCSyncStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartLsn int64 `protobuf:"varint,1,opt,name=start_lsn,json=startLsn,proto3" json:"start_lsn,omitempty"` + EndLsn int64 `protobuf:"varint,2,opt,name=end_lsn,json=endLsn,proto3" json:"end_lsn,omitempty"` + NumRows int32 `protobuf:"varint,3,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + StartTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` +} + +func (x *CDCSyncStatus) Reset() { + *x = CDCSyncStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CDCSyncStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CDCSyncStatus) ProtoMessage() {} + +func (x *CDCSyncStatus) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[15] + 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 CDCSyncStatus.ProtoReflect.Descriptor instead. +func (*CDCSyncStatus) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{15} +} + +func (x *CDCSyncStatus) GetStartLsn() int64 { + if x != nil { + return x.StartLsn + } + return 0 +} + +func (x *CDCSyncStatus) GetEndLsn() int64 { + if x != nil { + return x.EndLsn + } + return 0 +} + +func (x *CDCSyncStatus) GetNumRows() int32 { + if x != nil { + return x.NumRows + } + return 0 +} + +func (x *CDCSyncStatus) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *CDCSyncStatus) GetEndTime() *timestamppb.Timestamp { + if x != nil { + return x.EndTime + } + return nil +} + +type SnapshotStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Clones []*QRepMirrorStatus `protobuf:"bytes,1,rep,name=clones,proto3" json:"clones,omitempty"` +} + +func (x *SnapshotStatus) Reset() { + *x = SnapshotStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SnapshotStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SnapshotStatus) ProtoMessage() {} + +func (x *SnapshotStatus) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[16] + 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 SnapshotStatus.ProtoReflect.Descriptor instead. +func (*SnapshotStatus) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{16} +} + +func (x *SnapshotStatus) GetClones() []*QRepMirrorStatus { + if x != nil { + return x.Clones + } + return nil +} + +type CDCMirrorStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Config *FlowConnectionConfigs `protobuf:"bytes,1,opt,name=config,proto3" json:"config,omitempty"` + SnapshotStatus *SnapshotStatus `protobuf:"bytes,2,opt,name=snapshot_status,json=snapshotStatus,proto3" json:"snapshot_status,omitempty"` + CdcSyncs []*CDCSyncStatus `protobuf:"bytes,3,rep,name=cdc_syncs,json=cdcSyncs,proto3" json:"cdc_syncs,omitempty"` +} + +func (x *CDCMirrorStatus) Reset() { + *x = CDCMirrorStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CDCMirrorStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CDCMirrorStatus) ProtoMessage() {} + +func (x *CDCMirrorStatus) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[17] + 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 CDCMirrorStatus.ProtoReflect.Descriptor instead. +func (*CDCMirrorStatus) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{17} +} + +func (x *CDCMirrorStatus) GetConfig() *FlowConnectionConfigs { + if x != nil { + return x.Config + } + return nil +} + +func (x *CDCMirrorStatus) GetSnapshotStatus() *SnapshotStatus { + if x != nil { + return x.SnapshotStatus + } + return nil +} + +func (x *CDCMirrorStatus) GetCdcSyncs() []*CDCSyncStatus { + if x != nil { + return x.CdcSyncs + } + return nil +} + +type MirrorStatusResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FlowJobName string `protobuf:"bytes,1,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` + // Types that are assignable to Status: + // + // *MirrorStatusResponse_QrepStatus + // *MirrorStatusResponse_CdcStatus + Status isMirrorStatusResponse_Status `protobuf_oneof:"status"` + ErrorMessage string `protobuf:"bytes,4,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` +} + +func (x *MirrorStatusResponse) Reset() { + *x = MirrorStatusResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MirrorStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MirrorStatusResponse) ProtoMessage() {} + +func (x *MirrorStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[18] + 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 MirrorStatusResponse.ProtoReflect.Descriptor instead. +func (*MirrorStatusResponse) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{18} +} + +func (x *MirrorStatusResponse) GetFlowJobName() string { + if x != nil { + return x.FlowJobName + } + return "" +} + +func (m *MirrorStatusResponse) GetStatus() isMirrorStatusResponse_Status { + if m != nil { + return m.Status + } + return nil +} + +func (x *MirrorStatusResponse) GetQrepStatus() *QRepMirrorStatus { + if x, ok := x.GetStatus().(*MirrorStatusResponse_QrepStatus); ok { + return x.QrepStatus + } + return nil +} + +func (x *MirrorStatusResponse) GetCdcStatus() *CDCMirrorStatus { + if x, ok := x.GetStatus().(*MirrorStatusResponse_CdcStatus); ok { + return x.CdcStatus + } + return nil +} + +func (x *MirrorStatusResponse) GetErrorMessage() string { + if x != nil { + return x.ErrorMessage + } + return "" +} + +type isMirrorStatusResponse_Status interface { + isMirrorStatusResponse_Status() +} + +type MirrorStatusResponse_QrepStatus struct { + QrepStatus *QRepMirrorStatus `protobuf:"bytes,2,opt,name=qrep_status,json=qrepStatus,proto3,oneof"` +} + +type MirrorStatusResponse_CdcStatus struct { + CdcStatus *CDCMirrorStatus `protobuf:"bytes,3,opt,name=cdc_status,json=cdcStatus,proto3,oneof"` +} + +func (*MirrorStatusResponse_QrepStatus) isMirrorStatusResponse_Status() {} + +func (*MirrorStatusResponse_CdcStatus) isMirrorStatusResponse_Status() {} + var File_route_proto protoreflect.FileDescriptor var file_route_proto_rawDesc = []byte{ @@ -804,57 +1263,133 @@ var file_route_proto_rawDesc = []byte{ 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x2a, 0x42, 0x0a, 0x12, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x52, 0x45, 0x41, 0x54, 0x49, - 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, - 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x56, 0x41, 0x4c, - 0x49, 0x44, 0x10, 0x02, 0x2a, 0x43, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, - 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x12, 0x56, 0x41, 0x4c, 0x49, - 0x44, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, - 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0a, 0x0a, - 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x32, 0x95, 0x04, 0x0a, 0x0b, 0x46, 0x6c, - 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x09, 0x4c, 0x69, 0x73, - 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0c, 0x56, 0x61, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x70, + 0x22, 0x39, 0x0a, 0x13, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc1, 0x01, 0x0a, 0x0f, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x35, 0x0a, + 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, 0x65, 0x6e, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x22, + 0x82, 0x01, 0x0a, 0x10, 0x51, 0x52, 0x65, 0x70, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x2f, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3d, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xd2, 0x01, 0x0a, 0x0d, 0x43, 0x44, 0x43, 0x53, 0x79, 0x6e, 0x63, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, + 0x6c, 0x73, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x4c, 0x73, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x65, 0x6e, 0x64, 0x5f, 0x6c, 0x73, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x4c, 0x73, 0x6e, 0x12, 0x19, 0x0a, 0x08, + 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, + 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, + 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x48, 0x0a, 0x0e, 0x53, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x36, 0x0a, 0x06, 0x63, + 0x6c, 0x6f, 0x6e, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x4d, + 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x63, 0x6c, 0x6f, + 0x6e, 0x65, 0x73, 0x22, 0xce, 0x01, 0x0a, 0x0f, 0x43, 0x44, 0x43, 0x4d, 0x69, 0x72, 0x72, 0x6f, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3a, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x12, 0x45, 0x0a, 0x0f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0e, 0x73, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x63, 0x64, + 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x44, 0x43, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x08, 0x63, 0x64, 0x63, 0x53, + 0x79, 0x6e, 0x63, 0x73, 0x22, 0xec, 0x01, 0x0a, 0x14, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, + 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x41, 0x0a, 0x0b, 0x71, 0x72, 0x65, 0x70, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x0a, 0x71, 0x72, 0x65, 0x70, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x3e, 0x0a, 0x0a, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x44, 0x43, 0x4d, 0x69, 0x72, 0x72, 0x6f, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x09, 0x63, 0x64, 0x63, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x2a, 0x42, 0x0a, 0x12, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x50, + 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x52, 0x45, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, + 0x09, 0x0a, 0x05, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, + 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x02, 0x2a, 0x43, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x12, 0x56, + 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, + 0x4e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x44, 0x10, 0x01, + 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x32, 0xee, 0x04, 0x0a, + 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x09, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0c, + 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x56, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, - 0x12, 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x20, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, - 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, - 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, - 0x6c, 0x6f, 0x77, 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, - 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, - 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, - 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x4f, 0x0a, 0x0c, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x46, 0x6c, 0x6f, 0x77, - 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, - 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, - 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x72, 0x6f, 0x75, 0x74, 0x65, 0x42, 0x0a, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, - 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x52, 0x6f, 0x75, 0x74, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x56, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, + 0x65, 0x65, 0x72, 0x12, 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, + 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, + 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, + 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, + 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, + 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x46, + 0x6c, 0x6f, 0x77, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, + 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, + 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0c, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, + 0x75, 0x74, 0x65, 0x2e, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x7c, 0x0a, + 0x10, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, + 0x65, 0x42, 0x0a, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x52, 0x6f, 0x75, 0x74, 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, + 0x75, 0x74, 0x65, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, + 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, + 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -870,7 +1405,7 @@ func file_route_proto_rawDescGZIP() []byte { } var file_route_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_route_proto_msgTypes = make([]protoimpl.MessageInfo, 12) +var file_route_proto_msgTypes = make([]protoimpl.MessageInfo, 19) var file_route_proto_goTypes = []interface{}{ (ValidatePeerStatus)(0), // 0: peerdb_route.ValidatePeerStatus (CreatePeerStatus)(0), // 1: peerdb_route.CreatePeerStatus @@ -886,37 +1421,59 @@ var file_route_proto_goTypes = []interface{}{ (*CreatePeerRequest)(nil), // 11: peerdb_route.CreatePeerRequest (*ValidatePeerResponse)(nil), // 12: peerdb_route.ValidatePeerResponse (*CreatePeerResponse)(nil), // 13: peerdb_route.CreatePeerResponse - (*FlowConnectionConfigs)(nil), // 14: peerdb_flow.FlowConnectionConfigs - (*QRepConfig)(nil), // 15: peerdb_flow.QRepConfig - (*Peer)(nil), // 16: peerdb_peers.Peer + (*MirrorStatusRequest)(nil), // 14: peerdb_route.MirrorStatusRequest + (*PartitionStatus)(nil), // 15: peerdb_route.PartitionStatus + (*QRepMirrorStatus)(nil), // 16: peerdb_route.QRepMirrorStatus + (*CDCSyncStatus)(nil), // 17: peerdb_route.CDCSyncStatus + (*SnapshotStatus)(nil), // 18: peerdb_route.SnapshotStatus + (*CDCMirrorStatus)(nil), // 19: peerdb_route.CDCMirrorStatus + (*MirrorStatusResponse)(nil), // 20: peerdb_route.MirrorStatusResponse + (*FlowConnectionConfigs)(nil), // 21: peerdb_flow.FlowConnectionConfigs + (*QRepConfig)(nil), // 22: peerdb_flow.QRepConfig + (*Peer)(nil), // 23: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 24: google.protobuf.Timestamp } var file_route_proto_depIdxs = []int32{ - 14, // 0: peerdb_route.CreateCDCFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 15, // 1: peerdb_route.CreateQRepFlowRequest.qrep_config:type_name -> peerdb_flow.QRepConfig - 16, // 2: peerdb_route.ShutdownRequest.source_peer:type_name -> peerdb_peers.Peer - 16, // 3: peerdb_route.ShutdownRequest.destination_peer:type_name -> peerdb_peers.Peer - 16, // 4: peerdb_route.ListPeersResponse.peers:type_name -> peerdb_peers.Peer - 16, // 5: peerdb_route.ValidatePeerRequest.peer:type_name -> peerdb_peers.Peer - 16, // 6: peerdb_route.CreatePeerRequest.peer:type_name -> peerdb_peers.Peer + 21, // 0: peerdb_route.CreateCDCFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 22, // 1: peerdb_route.CreateQRepFlowRequest.qrep_config:type_name -> peerdb_flow.QRepConfig + 23, // 2: peerdb_route.ShutdownRequest.source_peer:type_name -> peerdb_peers.Peer + 23, // 3: peerdb_route.ShutdownRequest.destination_peer:type_name -> peerdb_peers.Peer + 23, // 4: peerdb_route.ListPeersResponse.peers:type_name -> peerdb_peers.Peer + 23, // 5: peerdb_route.ValidatePeerRequest.peer:type_name -> peerdb_peers.Peer + 23, // 6: peerdb_route.CreatePeerRequest.peer:type_name -> peerdb_peers.Peer 0, // 7: peerdb_route.ValidatePeerResponse.status:type_name -> peerdb_route.ValidatePeerStatus 1, // 8: peerdb_route.CreatePeerResponse.status:type_name -> peerdb_route.CreatePeerStatus - 8, // 9: peerdb_route.FlowService.ListPeers:input_type -> peerdb_route.ListPeersRequest - 10, // 10: peerdb_route.FlowService.ValidatePeer:input_type -> peerdb_route.ValidatePeerRequest - 11, // 11: peerdb_route.FlowService.CreatePeer:input_type -> peerdb_route.CreatePeerRequest - 2, // 12: peerdb_route.FlowService.CreateCDCFlow:input_type -> peerdb_route.CreateCDCFlowRequest - 4, // 13: peerdb_route.FlowService.CreateQRepFlow:input_type -> peerdb_route.CreateQRepFlowRequest - 6, // 14: peerdb_route.FlowService.ShutdownFlow:input_type -> peerdb_route.ShutdownRequest - 9, // 15: peerdb_route.FlowService.ListPeers:output_type -> peerdb_route.ListPeersResponse - 12, // 16: peerdb_route.FlowService.ValidatePeer:output_type -> peerdb_route.ValidatePeerResponse - 13, // 17: peerdb_route.FlowService.CreatePeer:output_type -> peerdb_route.CreatePeerResponse - 3, // 18: peerdb_route.FlowService.CreateCDCFlow:output_type -> peerdb_route.CreateCDCFlowResponse - 5, // 19: peerdb_route.FlowService.CreateQRepFlow:output_type -> peerdb_route.CreateQRepFlowResponse - 7, // 20: peerdb_route.FlowService.ShutdownFlow:output_type -> peerdb_route.ShutdownResponse - 15, // [15:21] is the sub-list for method output_type - 9, // [9:15] is the sub-list for method input_type - 9, // [9:9] is the sub-list for extension type_name - 9, // [9:9] is the sub-list for extension extendee - 0, // [0:9] is the sub-list for field type_name + 24, // 9: peerdb_route.PartitionStatus.start_time:type_name -> google.protobuf.Timestamp + 24, // 10: peerdb_route.PartitionStatus.end_time:type_name -> google.protobuf.Timestamp + 22, // 11: peerdb_route.QRepMirrorStatus.config:type_name -> peerdb_flow.QRepConfig + 15, // 12: peerdb_route.QRepMirrorStatus.partitions:type_name -> peerdb_route.PartitionStatus + 24, // 13: peerdb_route.CDCSyncStatus.start_time:type_name -> google.protobuf.Timestamp + 24, // 14: peerdb_route.CDCSyncStatus.end_time:type_name -> google.protobuf.Timestamp + 16, // 15: peerdb_route.SnapshotStatus.clones:type_name -> peerdb_route.QRepMirrorStatus + 21, // 16: peerdb_route.CDCMirrorStatus.config:type_name -> peerdb_flow.FlowConnectionConfigs + 18, // 17: peerdb_route.CDCMirrorStatus.snapshot_status:type_name -> peerdb_route.SnapshotStatus + 17, // 18: peerdb_route.CDCMirrorStatus.cdc_syncs:type_name -> peerdb_route.CDCSyncStatus + 16, // 19: peerdb_route.MirrorStatusResponse.qrep_status:type_name -> peerdb_route.QRepMirrorStatus + 19, // 20: peerdb_route.MirrorStatusResponse.cdc_status:type_name -> peerdb_route.CDCMirrorStatus + 8, // 21: peerdb_route.FlowService.ListPeers:input_type -> peerdb_route.ListPeersRequest + 10, // 22: peerdb_route.FlowService.ValidatePeer:input_type -> peerdb_route.ValidatePeerRequest + 11, // 23: peerdb_route.FlowService.CreatePeer:input_type -> peerdb_route.CreatePeerRequest + 2, // 24: peerdb_route.FlowService.CreateCDCFlow:input_type -> peerdb_route.CreateCDCFlowRequest + 4, // 25: peerdb_route.FlowService.CreateQRepFlow:input_type -> peerdb_route.CreateQRepFlowRequest + 6, // 26: peerdb_route.FlowService.ShutdownFlow:input_type -> peerdb_route.ShutdownRequest + 14, // 27: peerdb_route.FlowService.MirrorStatus:input_type -> peerdb_route.MirrorStatusRequest + 9, // 28: peerdb_route.FlowService.ListPeers:output_type -> peerdb_route.ListPeersResponse + 12, // 29: peerdb_route.FlowService.ValidatePeer:output_type -> peerdb_route.ValidatePeerResponse + 13, // 30: peerdb_route.FlowService.CreatePeer:output_type -> peerdb_route.CreatePeerResponse + 3, // 31: peerdb_route.FlowService.CreateCDCFlow:output_type -> peerdb_route.CreateCDCFlowResponse + 5, // 32: peerdb_route.FlowService.CreateQRepFlow:output_type -> peerdb_route.CreateQRepFlowResponse + 7, // 33: peerdb_route.FlowService.ShutdownFlow:output_type -> peerdb_route.ShutdownResponse + 20, // 34: peerdb_route.FlowService.MirrorStatus:output_type -> peerdb_route.MirrorStatusResponse + 28, // [28:35] is the sub-list for method output_type + 21, // [21:28] is the sub-list for method input_type + 21, // [21:21] is the sub-list for extension type_name + 21, // [21:21] is the sub-list for extension extendee + 0, // [0:21] is the sub-list for field type_name } func init() { file_route_proto_init() } @@ -1071,6 +1628,94 @@ func file_route_proto_init() { return nil } } + file_route_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MirrorStatusRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PartitionStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepMirrorStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CDCSyncStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SnapshotStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CDCMirrorStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MirrorStatusResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_route_proto_msgTypes[18].OneofWrappers = []interface{}{ + (*MirrorStatusResponse_QrepStatus)(nil), + (*MirrorStatusResponse_CdcStatus)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -1078,7 +1723,7 @@ func file_route_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_route_proto_rawDesc, NumEnums: 2, - NumMessages: 12, + NumMessages: 19, NumExtensions: 0, NumServices: 1, }, diff --git a/flow/generated/protos/route_grpc.pb.go b/flow/generated/protos/route_grpc.pb.go index ca9e1662bb..74ce7a6b0d 100644 --- a/flow/generated/protos/route_grpc.pb.go +++ b/flow/generated/protos/route_grpc.pb.go @@ -25,6 +25,7 @@ const ( FlowService_CreateCDCFlow_FullMethodName = "/peerdb_route.FlowService/CreateCDCFlow" FlowService_CreateQRepFlow_FullMethodName = "/peerdb_route.FlowService/CreateQRepFlow" FlowService_ShutdownFlow_FullMethodName = "/peerdb_route.FlowService/ShutdownFlow" + FlowService_MirrorStatus_FullMethodName = "/peerdb_route.FlowService/MirrorStatus" ) // FlowServiceClient is the client API for FlowService service. @@ -37,6 +38,7 @@ type FlowServiceClient interface { CreateCDCFlow(ctx context.Context, in *CreateCDCFlowRequest, opts ...grpc.CallOption) (*CreateCDCFlowResponse, error) CreateQRepFlow(ctx context.Context, in *CreateQRepFlowRequest, opts ...grpc.CallOption) (*CreateQRepFlowResponse, error) ShutdownFlow(ctx context.Context, in *ShutdownRequest, opts ...grpc.CallOption) (*ShutdownResponse, error) + MirrorStatus(ctx context.Context, in *MirrorStatusRequest, opts ...grpc.CallOption) (*MirrorStatusResponse, error) } type flowServiceClient struct { @@ -101,6 +103,15 @@ func (c *flowServiceClient) ShutdownFlow(ctx context.Context, in *ShutdownReques return out, nil } +func (c *flowServiceClient) MirrorStatus(ctx context.Context, in *MirrorStatusRequest, opts ...grpc.CallOption) (*MirrorStatusResponse, error) { + out := new(MirrorStatusResponse) + err := c.cc.Invoke(ctx, FlowService_MirrorStatus_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // FlowServiceServer is the server API for FlowService service. // All implementations must embed UnimplementedFlowServiceServer // for forward compatibility @@ -111,6 +122,7 @@ type FlowServiceServer interface { CreateCDCFlow(context.Context, *CreateCDCFlowRequest) (*CreateCDCFlowResponse, error) CreateQRepFlow(context.Context, *CreateQRepFlowRequest) (*CreateQRepFlowResponse, error) ShutdownFlow(context.Context, *ShutdownRequest) (*ShutdownResponse, error) + MirrorStatus(context.Context, *MirrorStatusRequest) (*MirrorStatusResponse, error) mustEmbedUnimplementedFlowServiceServer() } @@ -136,6 +148,9 @@ func (UnimplementedFlowServiceServer) CreateQRepFlow(context.Context, *CreateQRe func (UnimplementedFlowServiceServer) ShutdownFlow(context.Context, *ShutdownRequest) (*ShutdownResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ShutdownFlow not implemented") } +func (UnimplementedFlowServiceServer) MirrorStatus(context.Context, *MirrorStatusRequest) (*MirrorStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method MirrorStatus not implemented") +} func (UnimplementedFlowServiceServer) mustEmbedUnimplementedFlowServiceServer() {} // UnsafeFlowServiceServer may be embedded to opt out of forward compatibility for this service. @@ -257,6 +272,24 @@ func _FlowService_ShutdownFlow_Handler(srv interface{}, ctx context.Context, dec return interceptor(ctx, in, info, handler) } +func _FlowService_MirrorStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(MirrorStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(FlowServiceServer).MirrorStatus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: FlowService_MirrorStatus_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(FlowServiceServer).MirrorStatus(ctx, req.(*MirrorStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + // FlowService_ServiceDesc is the grpc.ServiceDesc for FlowService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -288,6 +321,10 @@ var FlowService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ShutdownFlow", Handler: _FlowService_ShutdownFlow_Handler, }, + { + MethodName: "MirrorStatus", + Handler: _FlowService_MirrorStatus_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "route.proto", diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 9f895586a8..d8ff8f232b 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -245,7 +245,10 @@ func SnapshotFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionCon numTablesInParallel = 1 } + logger.Info("cloning tables in parallel: ", numTablesInParallel) se.cloneTables(ctx, slotInfo, numTablesInParallel) + } else { + logger.Info("skipping initial copy as 'doInitialCopy' is false") } if err := se.closeSlotKeepAlive(replCtx); err != nil { diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 1945223d60..4da17cd4d7 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -176,7 +176,19 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { } let do_initial_copy = match raw_options.remove("do_initial_copy") { Some(sqlparser::ast::Value::Boolean(b)) => *b, - _ => false, + // also support "true" and "false" as strings + Some(sqlparser::ast::Value::SingleQuotedString(s)) => { + match s.as_ref() { + "true" => true, + "false" => false, + _ => { + return Err(anyhow::anyhow!( + "do_initial_copy must be a boolean" + )) + } + } + } + _ => return Err(anyhow::anyhow!("do_initial_copy must be a boolean")), }; let publication_name: Option = match raw_options diff --git a/nexus/catalog/migrations/V7__store_flow_config.sql b/nexus/catalog/migrations/V7__store_flow_config.sql new file mode 100644 index 0000000000..f9e6d4c0f6 --- /dev/null +++ b/nexus/catalog/migrations/V7__store_flow_config.sql @@ -0,0 +1,2 @@ +ALTER TABLE flows +ADD COLUMN config_proto BYTEA; diff --git a/nexus/pt/src/peerdb_route.rs b/nexus/pt/src/peerdb_route.rs index e07ebdc70c..bf0085ef8e 100644 --- a/nexus/pt/src/peerdb_route.rs +++ b/nexus/pt/src/peerdb_route.rs @@ -83,6 +83,85 @@ pub struct CreatePeerResponse { #[prost(string, tag="2")] pub message: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MirrorStatusRequest { + #[prost(string, tag="1")] + pub flow_job_name: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PartitionStatus { + #[prost(string, tag="1")] + pub partition_id: ::prost::alloc::string::String, + #[prost(message, optional, tag="2")] + pub start_time: ::core::option::Option<::pbjson_types::Timestamp>, + #[prost(message, optional, tag="3")] + pub end_time: ::core::option::Option<::pbjson_types::Timestamp>, + #[prost(int32, tag="4")] + pub num_rows: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct QRepMirrorStatus { + #[prost(message, optional, tag="1")] + pub config: ::core::option::Option, + /// TODO make note to see if we are still in initial copy + /// or if we are in the continuous streaming mode. + #[prost(message, repeated, tag="2")] + pub partitions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CdcSyncStatus { + #[prost(int64, tag="1")] + pub start_lsn: i64, + #[prost(int64, tag="2")] + pub end_lsn: i64, + #[prost(int32, tag="3")] + pub num_rows: i32, + #[prost(message, optional, tag="4")] + pub start_time: ::core::option::Option<::pbjson_types::Timestamp>, + #[prost(message, optional, tag="5")] + pub end_time: ::core::option::Option<::pbjson_types::Timestamp>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SnapshotStatus { + #[prost(message, repeated, tag="1")] + pub clones: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CdcMirrorStatus { + #[prost(message, optional, tag="1")] + pub config: ::core::option::Option, + #[prost(message, optional, tag="2")] + pub snapshot_status: ::core::option::Option, + #[prost(message, repeated, tag="3")] + pub cdc_syncs: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MirrorStatusResponse { + #[prost(string, tag="1")] + pub flow_job_name: ::prost::alloc::string::String, + #[prost(string, tag="4")] + pub error_message: ::prost::alloc::string::String, + #[prost(oneof="mirror_status_response::Status", tags="2, 3")] + pub status: ::core::option::Option, +} +/// Nested message and enum types in `MirrorStatusResponse`. +pub mod mirror_status_response { + #[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Status { + #[prost(message, tag="2")] + QrepStatus(super::QRepMirrorStatus), + #[prost(message, tag="3")] + CdcStatus(super::CdcMirrorStatus), + } +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum ValidatePeerStatus { diff --git a/nexus/pt/src/peerdb_route.serde.rs b/nexus/pt/src/peerdb_route.serde.rs index c3d59033f6..b6b8c2fc12 100644 --- a/nexus/pt/src/peerdb_route.serde.rs +++ b/nexus/pt/src/peerdb_route.serde.rs @@ -1,4 +1,309 @@ // @generated +impl serde::Serialize for CdcMirrorStatus { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.config.is_some() { + len += 1; + } + if self.snapshot_status.is_some() { + len += 1; + } + if !self.cdc_syncs.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.CDCMirrorStatus", len)?; + if let Some(v) = self.config.as_ref() { + struct_ser.serialize_field("config", v)?; + } + if let Some(v) = self.snapshot_status.as_ref() { + struct_ser.serialize_field("snapshotStatus", v)?; + } + if !self.cdc_syncs.is_empty() { + struct_ser.serialize_field("cdcSyncs", &self.cdc_syncs)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CdcMirrorStatus { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "config", + "snapshot_status", + "snapshotStatus", + "cdc_syncs", + "cdcSyncs", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Config, + SnapshotStatus, + CdcSyncs, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "config" => Ok(GeneratedField::Config), + "snapshotStatus" | "snapshot_status" => Ok(GeneratedField::SnapshotStatus), + "cdcSyncs" | "cdc_syncs" => Ok(GeneratedField::CdcSyncs), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CdcMirrorStatus; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.CDCMirrorStatus") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut config__ = None; + let mut snapshot_status__ = None; + let mut cdc_syncs__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Config => { + if config__.is_some() { + return Err(serde::de::Error::duplicate_field("config")); + } + config__ = map.next_value()?; + } + GeneratedField::SnapshotStatus => { + if snapshot_status__.is_some() { + return Err(serde::de::Error::duplicate_field("snapshotStatus")); + } + snapshot_status__ = map.next_value()?; + } + GeneratedField::CdcSyncs => { + if cdc_syncs__.is_some() { + return Err(serde::de::Error::duplicate_field("cdcSyncs")); + } + cdc_syncs__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(CdcMirrorStatus { + config: config__, + snapshot_status: snapshot_status__, + cdc_syncs: cdc_syncs__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_route.CDCMirrorStatus", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CdcSyncStatus { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.start_lsn != 0 { + len += 1; + } + if self.end_lsn != 0 { + len += 1; + } + if self.num_rows != 0 { + len += 1; + } + if self.start_time.is_some() { + len += 1; + } + if self.end_time.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.CDCSyncStatus", len)?; + if self.start_lsn != 0 { + struct_ser.serialize_field("startLsn", ToString::to_string(&self.start_lsn).as_str())?; + } + if self.end_lsn != 0 { + struct_ser.serialize_field("endLsn", ToString::to_string(&self.end_lsn).as_str())?; + } + if self.num_rows != 0 { + struct_ser.serialize_field("numRows", &self.num_rows)?; + } + if let Some(v) = self.start_time.as_ref() { + struct_ser.serialize_field("startTime", v)?; + } + if let Some(v) = self.end_time.as_ref() { + struct_ser.serialize_field("endTime", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CdcSyncStatus { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "start_lsn", + "startLsn", + "end_lsn", + "endLsn", + "num_rows", + "numRows", + "start_time", + "startTime", + "end_time", + "endTime", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + StartLsn, + EndLsn, + NumRows, + StartTime, + EndTime, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "startLsn" | "start_lsn" => Ok(GeneratedField::StartLsn), + "endLsn" | "end_lsn" => Ok(GeneratedField::EndLsn), + "numRows" | "num_rows" => Ok(GeneratedField::NumRows), + "startTime" | "start_time" => Ok(GeneratedField::StartTime), + "endTime" | "end_time" => Ok(GeneratedField::EndTime), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CdcSyncStatus; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.CDCSyncStatus") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut start_lsn__ = None; + let mut end_lsn__ = None; + let mut num_rows__ = None; + let mut start_time__ = None; + let mut end_time__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::StartLsn => { + if start_lsn__.is_some() { + return Err(serde::de::Error::duplicate_field("startLsn")); + } + start_lsn__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::EndLsn => { + if end_lsn__.is_some() { + return Err(serde::de::Error::duplicate_field("endLsn")); + } + end_lsn__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::NumRows => { + if num_rows__.is_some() { + return Err(serde::de::Error::duplicate_field("numRows")); + } + num_rows__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::StartTime => { + if start_time__.is_some() { + return Err(serde::de::Error::duplicate_field("startTime")); + } + start_time__ = map.next_value()?; + } + GeneratedField::EndTime => { + if end_time__.is_some() { + return Err(serde::de::Error::duplicate_field("endTime")); + } + end_time__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(CdcSyncStatus { + start_lsn: start_lsn__.unwrap_or_default(), + end_lsn: end_lsn__.unwrap_or_default(), + num_rows: num_rows__.unwrap_or_default(), + start_time: start_time__, + end_time: end_time__, + }) + } + } + deserializer.deserialize_struct("peerdb_route.CDCSyncStatus", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CreateCdcFlowRequest { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -853,7 +1158,7 @@ impl<'de> serde::Deserialize<'de> for ListPeersResponse { deserializer.deserialize_struct("peerdb_route.ListPeersResponse", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for ShutdownRequest { +impl serde::Serialize for MirrorStatusRequest { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -861,57 +1166,30 @@ impl serde::Serialize for ShutdownRequest { { use serde::ser::SerializeStruct; let mut len = 0; - if !self.workflow_id.is_empty() { - len += 1; - } if !self.flow_job_name.is_empty() { len += 1; } - if self.source_peer.is_some() { - len += 1; - } - if self.destination_peer.is_some() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("peerdb_route.ShutdownRequest", len)?; - if !self.workflow_id.is_empty() { - struct_ser.serialize_field("workflowId", &self.workflow_id)?; - } + let mut struct_ser = serializer.serialize_struct("peerdb_route.MirrorStatusRequest", len)?; if !self.flow_job_name.is_empty() { struct_ser.serialize_field("flowJobName", &self.flow_job_name)?; } - if let Some(v) = self.source_peer.as_ref() { - struct_ser.serialize_field("sourcePeer", v)?; - } - if let Some(v) = self.destination_peer.as_ref() { - struct_ser.serialize_field("destinationPeer", v)?; - } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for ShutdownRequest { +impl<'de> serde::Deserialize<'de> for MirrorStatusRequest { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "workflow_id", - "workflowId", "flow_job_name", "flowJobName", - "source_peer", - "sourcePeer", - "destination_peer", - "destinationPeer", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - WorkflowId, FlowJobName, - SourcePeer, - DestinationPeer, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -934,10 +1212,7 @@ impl<'de> serde::Deserialize<'de> for ShutdownRequest { E: serde::de::Error, { match value { - "workflowId" | "workflow_id" => Ok(GeneratedField::WorkflowId), "flowJobName" | "flow_job_name" => Ok(GeneratedField::FlowJobName), - "sourcePeer" | "source_peer" => Ok(GeneratedField::SourcePeer), - "destinationPeer" | "destination_peer" => Ok(GeneratedField::DestinationPeer), _ => Ok(GeneratedField::__SkipField__), } } @@ -947,63 +1222,39 @@ impl<'de> serde::Deserialize<'de> for ShutdownRequest { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = ShutdownRequest; + type Value = MirrorStatusRequest; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_route.ShutdownRequest") + formatter.write_str("struct peerdb_route.MirrorStatusRequest") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut workflow_id__ = None; let mut flow_job_name__ = None; - let mut source_peer__ = None; - let mut destination_peer__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::WorkflowId => { - if workflow_id__.is_some() { - return Err(serde::de::Error::duplicate_field("workflowId")); - } - workflow_id__ = Some(map.next_value()?); - } GeneratedField::FlowJobName => { if flow_job_name__.is_some() { return Err(serde::de::Error::duplicate_field("flowJobName")); } flow_job_name__ = Some(map.next_value()?); } - GeneratedField::SourcePeer => { - if source_peer__.is_some() { - return Err(serde::de::Error::duplicate_field("sourcePeer")); - } - source_peer__ = map.next_value()?; - } - GeneratedField::DestinationPeer => { - if destination_peer__.is_some() { - return Err(serde::de::Error::duplicate_field("destinationPeer")); - } - destination_peer__ = map.next_value()?; - } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } } } - Ok(ShutdownRequest { - workflow_id: workflow_id__.unwrap_or_default(), + Ok(MirrorStatusRequest { flow_job_name: flow_job_name__.unwrap_or_default(), - source_peer: source_peer__, - destination_peer: destination_peer__, }) } } - deserializer.deserialize_struct("peerdb_route.ShutdownRequest", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_route.MirrorStatusRequest", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for ShutdownResponse { +impl serde::Serialize for MirrorStatusResponse { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -1011,15 +1262,580 @@ impl serde::Serialize for ShutdownResponse { { use serde::ser::SerializeStruct; let mut len = 0; - if self.ok { + if !self.flow_job_name.is_empty() { len += 1; } if !self.error_message.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_route.ShutdownResponse", len)?; - if self.ok { - struct_ser.serialize_field("ok", &self.ok)?; + if self.status.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.MirrorStatusResponse", len)?; + if !self.flow_job_name.is_empty() { + struct_ser.serialize_field("flowJobName", &self.flow_job_name)?; + } + if !self.error_message.is_empty() { + struct_ser.serialize_field("errorMessage", &self.error_message)?; + } + if let Some(v) = self.status.as_ref() { + match v { + mirror_status_response::Status::QrepStatus(v) => { + struct_ser.serialize_field("qrepStatus", v)?; + } + mirror_status_response::Status::CdcStatus(v) => { + struct_ser.serialize_field("cdcStatus", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MirrorStatusResponse { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flow_job_name", + "flowJobName", + "error_message", + "errorMessage", + "qrep_status", + "qrepStatus", + "cdc_status", + "cdcStatus", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FlowJobName, + ErrorMessage, + QrepStatus, + CdcStatus, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "flowJobName" | "flow_job_name" => Ok(GeneratedField::FlowJobName), + "errorMessage" | "error_message" => Ok(GeneratedField::ErrorMessage), + "qrepStatus" | "qrep_status" => Ok(GeneratedField::QrepStatus), + "cdcStatus" | "cdc_status" => Ok(GeneratedField::CdcStatus), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MirrorStatusResponse; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.MirrorStatusResponse") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flow_job_name__ = None; + let mut error_message__ = None; + let mut status__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::FlowJobName => { + if flow_job_name__.is_some() { + return Err(serde::de::Error::duplicate_field("flowJobName")); + } + flow_job_name__ = Some(map.next_value()?); + } + GeneratedField::ErrorMessage => { + if error_message__.is_some() { + return Err(serde::de::Error::duplicate_field("errorMessage")); + } + error_message__ = Some(map.next_value()?); + } + GeneratedField::QrepStatus => { + if status__.is_some() { + return Err(serde::de::Error::duplicate_field("qrepStatus")); + } + status__ = map.next_value::<::std::option::Option<_>>()?.map(mirror_status_response::Status::QrepStatus) +; + } + GeneratedField::CdcStatus => { + if status__.is_some() { + return Err(serde::de::Error::duplicate_field("cdcStatus")); + } + status__ = map.next_value::<::std::option::Option<_>>()?.map(mirror_status_response::Status::CdcStatus) +; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(MirrorStatusResponse { + flow_job_name: flow_job_name__.unwrap_or_default(), + error_message: error_message__.unwrap_or_default(), + status: status__, + }) + } + } + deserializer.deserialize_struct("peerdb_route.MirrorStatusResponse", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for PartitionStatus { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.partition_id.is_empty() { + len += 1; + } + if self.start_time.is_some() { + len += 1; + } + if self.end_time.is_some() { + len += 1; + } + if self.num_rows != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.PartitionStatus", len)?; + if !self.partition_id.is_empty() { + struct_ser.serialize_field("partitionId", &self.partition_id)?; + } + if let Some(v) = self.start_time.as_ref() { + struct_ser.serialize_field("startTime", v)?; + } + if let Some(v) = self.end_time.as_ref() { + struct_ser.serialize_field("endTime", v)?; + } + if self.num_rows != 0 { + struct_ser.serialize_field("numRows", &self.num_rows)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PartitionStatus { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "partition_id", + "partitionId", + "start_time", + "startTime", + "end_time", + "endTime", + "num_rows", + "numRows", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + PartitionId, + StartTime, + EndTime, + NumRows, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "partitionId" | "partition_id" => Ok(GeneratedField::PartitionId), + "startTime" | "start_time" => Ok(GeneratedField::StartTime), + "endTime" | "end_time" => Ok(GeneratedField::EndTime), + "numRows" | "num_rows" => Ok(GeneratedField::NumRows), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PartitionStatus; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.PartitionStatus") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut partition_id__ = None; + let mut start_time__ = None; + let mut end_time__ = None; + let mut num_rows__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::PartitionId => { + if partition_id__.is_some() { + return Err(serde::de::Error::duplicate_field("partitionId")); + } + partition_id__ = Some(map.next_value()?); + } + GeneratedField::StartTime => { + if start_time__.is_some() { + return Err(serde::de::Error::duplicate_field("startTime")); + } + start_time__ = map.next_value()?; + } + GeneratedField::EndTime => { + if end_time__.is_some() { + return Err(serde::de::Error::duplicate_field("endTime")); + } + end_time__ = map.next_value()?; + } + GeneratedField::NumRows => { + if num_rows__.is_some() { + return Err(serde::de::Error::duplicate_field("numRows")); + } + num_rows__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(PartitionStatus { + partition_id: partition_id__.unwrap_or_default(), + start_time: start_time__, + end_time: end_time__, + num_rows: num_rows__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_route.PartitionStatus", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for QRepMirrorStatus { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.config.is_some() { + len += 1; + } + if !self.partitions.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.QRepMirrorStatus", len)?; + if let Some(v) = self.config.as_ref() { + struct_ser.serialize_field("config", v)?; + } + if !self.partitions.is_empty() { + struct_ser.serialize_field("partitions", &self.partitions)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for QRepMirrorStatus { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "config", + "partitions", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Config, + Partitions, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "config" => Ok(GeneratedField::Config), + "partitions" => Ok(GeneratedField::Partitions), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = QRepMirrorStatus; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.QRepMirrorStatus") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut config__ = None; + let mut partitions__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Config => { + if config__.is_some() { + return Err(serde::de::Error::duplicate_field("config")); + } + config__ = map.next_value()?; + } + GeneratedField::Partitions => { + if partitions__.is_some() { + return Err(serde::de::Error::duplicate_field("partitions")); + } + partitions__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(QRepMirrorStatus { + config: config__, + partitions: partitions__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_route.QRepMirrorStatus", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ShutdownRequest { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.workflow_id.is_empty() { + len += 1; + } + if !self.flow_job_name.is_empty() { + len += 1; + } + if self.source_peer.is_some() { + len += 1; + } + if self.destination_peer.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.ShutdownRequest", len)?; + if !self.workflow_id.is_empty() { + struct_ser.serialize_field("workflowId", &self.workflow_id)?; + } + if !self.flow_job_name.is_empty() { + struct_ser.serialize_field("flowJobName", &self.flow_job_name)?; + } + if let Some(v) = self.source_peer.as_ref() { + struct_ser.serialize_field("sourcePeer", v)?; + } + if let Some(v) = self.destination_peer.as_ref() { + struct_ser.serialize_field("destinationPeer", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ShutdownRequest { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "workflow_id", + "workflowId", + "flow_job_name", + "flowJobName", + "source_peer", + "sourcePeer", + "destination_peer", + "destinationPeer", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + WorkflowId, + FlowJobName, + SourcePeer, + DestinationPeer, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "workflowId" | "workflow_id" => Ok(GeneratedField::WorkflowId), + "flowJobName" | "flow_job_name" => Ok(GeneratedField::FlowJobName), + "sourcePeer" | "source_peer" => Ok(GeneratedField::SourcePeer), + "destinationPeer" | "destination_peer" => Ok(GeneratedField::DestinationPeer), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ShutdownRequest; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.ShutdownRequest") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut workflow_id__ = None; + let mut flow_job_name__ = None; + let mut source_peer__ = None; + let mut destination_peer__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::WorkflowId => { + if workflow_id__.is_some() { + return Err(serde::de::Error::duplicate_field("workflowId")); + } + workflow_id__ = Some(map.next_value()?); + } + GeneratedField::FlowJobName => { + if flow_job_name__.is_some() { + return Err(serde::de::Error::duplicate_field("flowJobName")); + } + flow_job_name__ = Some(map.next_value()?); + } + GeneratedField::SourcePeer => { + if source_peer__.is_some() { + return Err(serde::de::Error::duplicate_field("sourcePeer")); + } + source_peer__ = map.next_value()?; + } + GeneratedField::DestinationPeer => { + if destination_peer__.is_some() { + return Err(serde::de::Error::duplicate_field("destinationPeer")); + } + destination_peer__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(ShutdownRequest { + workflow_id: workflow_id__.unwrap_or_default(), + flow_job_name: flow_job_name__.unwrap_or_default(), + source_peer: source_peer__, + destination_peer: destination_peer__, + }) + } + } + deserializer.deserialize_struct("peerdb_route.ShutdownRequest", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ShutdownResponse { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.ok { + len += 1; + } + if !self.error_message.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.ShutdownResponse", len)?; + if self.ok { + struct_ser.serialize_field("ok", &self.ok)?; } if !self.error_message.is_empty() { struct_ser.serialize_field("errorMessage", &self.error_message)?; @@ -1116,6 +1932,101 @@ impl<'de> serde::Deserialize<'de> for ShutdownResponse { deserializer.deserialize_struct("peerdb_route.ShutdownResponse", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SnapshotStatus { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.clones.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.SnapshotStatus", len)?; + if !self.clones.is_empty() { + struct_ser.serialize_field("clones", &self.clones)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SnapshotStatus { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "clones", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Clones, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "clones" => Ok(GeneratedField::Clones), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SnapshotStatus; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.SnapshotStatus") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut clones__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Clones => { + if clones__.is_some() { + return Err(serde::de::Error::duplicate_field("clones")); + } + clones__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(SnapshotStatus { + clones: clones__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_route.SnapshotStatus", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ValidatePeerRequest { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/nexus/pt/src/peerdb_route.tonic.rs b/nexus/pt/src/peerdb_route.tonic.rs index 9c251014ec..bbebc8ab85 100644 --- a/nexus/pt/src/peerdb_route.tonic.rs +++ b/nexus/pt/src/peerdb_route.tonic.rs @@ -241,6 +241,32 @@ pub mod flow_service_client { .insert(GrpcMethod::new("peerdb_route.FlowService", "ShutdownFlow")); self.inner.unary(req, path, codec).await } + /// + pub async fn mirror_status( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/peerdb_route.FlowService/MirrorStatus", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("peerdb_route.FlowService", "MirrorStatus")); + self.inner.unary(req, path, codec).await + } } } /// Generated server implementations. @@ -298,6 +324,14 @@ pub mod flow_service_server { tonic::Response, tonic::Status, >; + /// + async fn mirror_status( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; } /// #[derive(Debug)] @@ -651,6 +685,52 @@ pub mod flow_service_server { }; Box::pin(fut) } + "/peerdb_route.FlowService/MirrorStatus" => { + #[allow(non_camel_case_types)] + struct MirrorStatusSvc(pub Arc); + impl< + T: FlowService, + > tonic::server::UnaryService + for MirrorStatusSvc { + type Response = super::MirrorStatusResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).mirror_status(request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = MirrorStatusSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } _ => { Box::pin(async move { Ok( diff --git a/protos/route.proto b/protos/route.proto index 37d3b530df..d0e4bea697 100644 --- a/protos/route.proto +++ b/protos/route.proto @@ -72,6 +72,51 @@ message CreatePeerResponse { string message = 2; } +message MirrorStatusRequest { + string flow_job_name = 1; +} + +message PartitionStatus { + string partition_id = 1; + google.protobuf.Timestamp start_time = 2; + google.protobuf.Timestamp end_time = 3; + int32 num_rows = 4; +} + +message QRepMirrorStatus { + peerdb_flow.QRepConfig config = 1; + repeated PartitionStatus partitions = 2; + // TODO make note to see if we are still in initial copy + // or if we are in the continuous streaming mode. +} + +message CDCSyncStatus { + int64 start_lsn = 1; + int64 end_lsn = 2; + int32 num_rows = 3; + google.protobuf.Timestamp start_time = 4; + google.protobuf.Timestamp end_time = 5; +} + +message SnapshotStatus { + repeated QRepMirrorStatus clones = 1; +} + +message CDCMirrorStatus { + peerdb_flow.FlowConnectionConfigs config = 1; + SnapshotStatus snapshot_status = 2; + repeated CDCSyncStatus cdc_syncs = 3; +} + +message MirrorStatusResponse { + string flow_job_name = 1; + oneof status { + QRepMirrorStatus qrep_status = 2; + CDCMirrorStatus cdc_status = 3; + } + string error_message = 4; +} + service FlowService { rpc ListPeers(ListPeersRequest) returns (ListPeersResponse) {} rpc ValidatePeer(ValidatePeerRequest) returns (ValidatePeerResponse) {} @@ -79,4 +124,5 @@ service FlowService { rpc CreateCDCFlow(CreateCDCFlowRequest) returns (CreateCDCFlowResponse) {} rpc CreateQRepFlow(CreateQRepFlowRequest) returns (CreateQRepFlowResponse) {} rpc ShutdownFlow(ShutdownRequest) returns (ShutdownResponse) {} + rpc MirrorStatus(MirrorStatusRequest) returns (MirrorStatusResponse) {} } diff --git a/ui/grpc_generated/route.ts b/ui/grpc_generated/route.ts index 6874a48d93..df68fcd19c 100644 --- a/ui/grpc_generated/route.ts +++ b/ui/grpc_generated/route.ts @@ -11,8 +11,10 @@ import { ServiceError, UntypedServiceImplementation, } from "@grpc/grpc-js"; +import Long from "long"; import _m0 from "protobufjs/minimal"; import { FlowConnectionConfigs, QRepConfig } from "./flow"; +import { Timestamp } from "./google/protobuf/timestamp"; import { Peer } from "./peers"; export const protobufPackage = "peerdb_route"; @@ -149,6 +151,53 @@ export interface CreatePeerResponse { message: string; } +export interface MirrorStatusRequest { + flowJobName: string; +} + +export interface PartitionStatus { + partitionId: string; + startTime: Date | undefined; + endTime: Date | undefined; + numRows: number; +} + +export interface QRepMirrorStatus { + config: + | QRepConfig + | undefined; + /** + * TODO make note to see if we are still in initial copy + * or if we are in the continuous streaming mode. + */ + partitions: PartitionStatus[]; +} + +export interface CDCSyncStatus { + startLsn: number; + endLsn: number; + numRows: number; + startTime: Date | undefined; + endTime: Date | undefined; +} + +export interface SnapshotStatus { + clones: QRepMirrorStatus[]; +} + +export interface CDCMirrorStatus { + config: FlowConnectionConfigs | undefined; + snapshotStatus: SnapshotStatus | undefined; + cdcSyncs: CDCSyncStatus[]; +} + +export interface MirrorStatusResponse { + flowJobName: string; + qrepStatus?: QRepMirrorStatus | undefined; + cdcStatus?: CDCMirrorStatus | undefined; + errorMessage: string; +} + function createBaseCreateCDCFlowRequest(): CreateCDCFlowRequest { return { connectionConfigs: undefined, createCatalogEntry: false }; } @@ -944,6 +993,622 @@ export const CreatePeerResponse = { }, }; +function createBaseMirrorStatusRequest(): MirrorStatusRequest { + return { flowJobName: "" }; +} + +export const MirrorStatusRequest = { + encode(message: MirrorStatusRequest, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.flowJobName !== "") { + writer.uint32(10).string(message.flowJobName); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): MirrorStatusRequest { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseMirrorStatusRequest(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.flowJobName = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): MirrorStatusRequest { + return { flowJobName: isSet(object.flowJobName) ? String(object.flowJobName) : "" }; + }, + + toJSON(message: MirrorStatusRequest): unknown { + const obj: any = {}; + if (message.flowJobName !== "") { + obj.flowJobName = message.flowJobName; + } + return obj; + }, + + create, I>>(base?: I): MirrorStatusRequest { + return MirrorStatusRequest.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): MirrorStatusRequest { + const message = createBaseMirrorStatusRequest(); + message.flowJobName = object.flowJobName ?? ""; + return message; + }, +}; + +function createBasePartitionStatus(): PartitionStatus { + return { partitionId: "", startTime: undefined, endTime: undefined, numRows: 0 }; +} + +export const PartitionStatus = { + encode(message: PartitionStatus, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.partitionId !== "") { + writer.uint32(10).string(message.partitionId); + } + if (message.startTime !== undefined) { + Timestamp.encode(toTimestamp(message.startTime), writer.uint32(18).fork()).ldelim(); + } + if (message.endTime !== undefined) { + Timestamp.encode(toTimestamp(message.endTime), writer.uint32(26).fork()).ldelim(); + } + if (message.numRows !== 0) { + writer.uint32(32).int32(message.numRows); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): PartitionStatus { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePartitionStatus(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.partitionId = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.startTime = fromTimestamp(Timestamp.decode(reader, reader.uint32())); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.endTime = fromTimestamp(Timestamp.decode(reader, reader.uint32())); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.numRows = reader.int32(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): PartitionStatus { + return { + partitionId: isSet(object.partitionId) ? String(object.partitionId) : "", + startTime: isSet(object.startTime) ? fromJsonTimestamp(object.startTime) : undefined, + endTime: isSet(object.endTime) ? fromJsonTimestamp(object.endTime) : undefined, + numRows: isSet(object.numRows) ? Number(object.numRows) : 0, + }; + }, + + toJSON(message: PartitionStatus): unknown { + const obj: any = {}; + if (message.partitionId !== "") { + obj.partitionId = message.partitionId; + } + if (message.startTime !== undefined) { + obj.startTime = message.startTime.toISOString(); + } + if (message.endTime !== undefined) { + obj.endTime = message.endTime.toISOString(); + } + if (message.numRows !== 0) { + obj.numRows = Math.round(message.numRows); + } + return obj; + }, + + create, I>>(base?: I): PartitionStatus { + return PartitionStatus.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PartitionStatus { + const message = createBasePartitionStatus(); + message.partitionId = object.partitionId ?? ""; + message.startTime = object.startTime ?? undefined; + message.endTime = object.endTime ?? undefined; + message.numRows = object.numRows ?? 0; + return message; + }, +}; + +function createBaseQRepMirrorStatus(): QRepMirrorStatus { + return { config: undefined, partitions: [] }; +} + +export const QRepMirrorStatus = { + encode(message: QRepMirrorStatus, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.config !== undefined) { + QRepConfig.encode(message.config, writer.uint32(10).fork()).ldelim(); + } + for (const v of message.partitions) { + PartitionStatus.encode(v!, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): QRepMirrorStatus { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseQRepMirrorStatus(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.config = QRepConfig.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.partitions.push(PartitionStatus.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): QRepMirrorStatus { + return { + config: isSet(object.config) ? QRepConfig.fromJSON(object.config) : undefined, + partitions: Array.isArray(object?.partitions) + ? object.partitions.map((e: any) => PartitionStatus.fromJSON(e)) + : [], + }; + }, + + toJSON(message: QRepMirrorStatus): unknown { + const obj: any = {}; + if (message.config !== undefined) { + obj.config = QRepConfig.toJSON(message.config); + } + if (message.partitions?.length) { + obj.partitions = message.partitions.map((e) => PartitionStatus.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): QRepMirrorStatus { + return QRepMirrorStatus.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): QRepMirrorStatus { + const message = createBaseQRepMirrorStatus(); + message.config = (object.config !== undefined && object.config !== null) + ? QRepConfig.fromPartial(object.config) + : undefined; + message.partitions = object.partitions?.map((e) => PartitionStatus.fromPartial(e)) || []; + return message; + }, +}; + +function createBaseCDCSyncStatus(): CDCSyncStatus { + return { startLsn: 0, endLsn: 0, numRows: 0, startTime: undefined, endTime: undefined }; +} + +export const CDCSyncStatus = { + encode(message: CDCSyncStatus, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.startLsn !== 0) { + writer.uint32(8).int64(message.startLsn); + } + if (message.endLsn !== 0) { + writer.uint32(16).int64(message.endLsn); + } + if (message.numRows !== 0) { + writer.uint32(24).int32(message.numRows); + } + if (message.startTime !== undefined) { + Timestamp.encode(toTimestamp(message.startTime), writer.uint32(34).fork()).ldelim(); + } + if (message.endTime !== undefined) { + Timestamp.encode(toTimestamp(message.endTime), writer.uint32(42).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): CDCSyncStatus { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseCDCSyncStatus(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.startLsn = longToNumber(reader.int64() as Long); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.endLsn = longToNumber(reader.int64() as Long); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.numRows = reader.int32(); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.startTime = fromTimestamp(Timestamp.decode(reader, reader.uint32())); + continue; + case 5: + if (tag !== 42) { + break; + } + + message.endTime = fromTimestamp(Timestamp.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): CDCSyncStatus { + return { + startLsn: isSet(object.startLsn) ? Number(object.startLsn) : 0, + endLsn: isSet(object.endLsn) ? Number(object.endLsn) : 0, + numRows: isSet(object.numRows) ? Number(object.numRows) : 0, + startTime: isSet(object.startTime) ? fromJsonTimestamp(object.startTime) : undefined, + endTime: isSet(object.endTime) ? fromJsonTimestamp(object.endTime) : undefined, + }; + }, + + toJSON(message: CDCSyncStatus): unknown { + const obj: any = {}; + if (message.startLsn !== 0) { + obj.startLsn = Math.round(message.startLsn); + } + if (message.endLsn !== 0) { + obj.endLsn = Math.round(message.endLsn); + } + if (message.numRows !== 0) { + obj.numRows = Math.round(message.numRows); + } + if (message.startTime !== undefined) { + obj.startTime = message.startTime.toISOString(); + } + if (message.endTime !== undefined) { + obj.endTime = message.endTime.toISOString(); + } + return obj; + }, + + create, I>>(base?: I): CDCSyncStatus { + return CDCSyncStatus.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): CDCSyncStatus { + const message = createBaseCDCSyncStatus(); + message.startLsn = object.startLsn ?? 0; + message.endLsn = object.endLsn ?? 0; + message.numRows = object.numRows ?? 0; + message.startTime = object.startTime ?? undefined; + message.endTime = object.endTime ?? undefined; + return message; + }, +}; + +function createBaseSnapshotStatus(): SnapshotStatus { + return { clones: [] }; +} + +export const SnapshotStatus = { + encode(message: SnapshotStatus, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + for (const v of message.clones) { + QRepMirrorStatus.encode(v!, writer.uint32(10).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SnapshotStatus { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSnapshotStatus(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.clones.push(QRepMirrorStatus.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SnapshotStatus { + return { clones: Array.isArray(object?.clones) ? object.clones.map((e: any) => QRepMirrorStatus.fromJSON(e)) : [] }; + }, + + toJSON(message: SnapshotStatus): unknown { + const obj: any = {}; + if (message.clones?.length) { + obj.clones = message.clones.map((e) => QRepMirrorStatus.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): SnapshotStatus { + return SnapshotStatus.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): SnapshotStatus { + const message = createBaseSnapshotStatus(); + message.clones = object.clones?.map((e) => QRepMirrorStatus.fromPartial(e)) || []; + return message; + }, +}; + +function createBaseCDCMirrorStatus(): CDCMirrorStatus { + return { config: undefined, snapshotStatus: undefined, cdcSyncs: [] }; +} + +export const CDCMirrorStatus = { + encode(message: CDCMirrorStatus, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.config !== undefined) { + FlowConnectionConfigs.encode(message.config, writer.uint32(10).fork()).ldelim(); + } + if (message.snapshotStatus !== undefined) { + SnapshotStatus.encode(message.snapshotStatus, writer.uint32(18).fork()).ldelim(); + } + for (const v of message.cdcSyncs) { + CDCSyncStatus.encode(v!, writer.uint32(26).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): CDCMirrorStatus { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseCDCMirrorStatus(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.config = FlowConnectionConfigs.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.snapshotStatus = SnapshotStatus.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.cdcSyncs.push(CDCSyncStatus.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): CDCMirrorStatus { + return { + config: isSet(object.config) ? FlowConnectionConfigs.fromJSON(object.config) : undefined, + snapshotStatus: isSet(object.snapshotStatus) ? SnapshotStatus.fromJSON(object.snapshotStatus) : undefined, + cdcSyncs: Array.isArray(object?.cdcSyncs) ? object.cdcSyncs.map((e: any) => CDCSyncStatus.fromJSON(e)) : [], + }; + }, + + toJSON(message: CDCMirrorStatus): unknown { + const obj: any = {}; + if (message.config !== undefined) { + obj.config = FlowConnectionConfigs.toJSON(message.config); + } + if (message.snapshotStatus !== undefined) { + obj.snapshotStatus = SnapshotStatus.toJSON(message.snapshotStatus); + } + if (message.cdcSyncs?.length) { + obj.cdcSyncs = message.cdcSyncs.map((e) => CDCSyncStatus.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): CDCMirrorStatus { + return CDCMirrorStatus.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): CDCMirrorStatus { + const message = createBaseCDCMirrorStatus(); + message.config = (object.config !== undefined && object.config !== null) + ? FlowConnectionConfigs.fromPartial(object.config) + : undefined; + message.snapshotStatus = (object.snapshotStatus !== undefined && object.snapshotStatus !== null) + ? SnapshotStatus.fromPartial(object.snapshotStatus) + : undefined; + message.cdcSyncs = object.cdcSyncs?.map((e) => CDCSyncStatus.fromPartial(e)) || []; + return message; + }, +}; + +function createBaseMirrorStatusResponse(): MirrorStatusResponse { + return { flowJobName: "", qrepStatus: undefined, cdcStatus: undefined, errorMessage: "" }; +} + +export const MirrorStatusResponse = { + encode(message: MirrorStatusResponse, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.flowJobName !== "") { + writer.uint32(10).string(message.flowJobName); + } + if (message.qrepStatus !== undefined) { + QRepMirrorStatus.encode(message.qrepStatus, writer.uint32(18).fork()).ldelim(); + } + if (message.cdcStatus !== undefined) { + CDCMirrorStatus.encode(message.cdcStatus, writer.uint32(26).fork()).ldelim(); + } + if (message.errorMessage !== "") { + writer.uint32(34).string(message.errorMessage); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): MirrorStatusResponse { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseMirrorStatusResponse(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.flowJobName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.qrepStatus = QRepMirrorStatus.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.cdcStatus = CDCMirrorStatus.decode(reader, reader.uint32()); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.errorMessage = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): MirrorStatusResponse { + return { + flowJobName: isSet(object.flowJobName) ? String(object.flowJobName) : "", + qrepStatus: isSet(object.qrepStatus) ? QRepMirrorStatus.fromJSON(object.qrepStatus) : undefined, + cdcStatus: isSet(object.cdcStatus) ? CDCMirrorStatus.fromJSON(object.cdcStatus) : undefined, + errorMessage: isSet(object.errorMessage) ? String(object.errorMessage) : "", + }; + }, + + toJSON(message: MirrorStatusResponse): unknown { + const obj: any = {}; + if (message.flowJobName !== "") { + obj.flowJobName = message.flowJobName; + } + if (message.qrepStatus !== undefined) { + obj.qrepStatus = QRepMirrorStatus.toJSON(message.qrepStatus); + } + if (message.cdcStatus !== undefined) { + obj.cdcStatus = CDCMirrorStatus.toJSON(message.cdcStatus); + } + if (message.errorMessage !== "") { + obj.errorMessage = message.errorMessage; + } + return obj; + }, + + create, I>>(base?: I): MirrorStatusResponse { + return MirrorStatusResponse.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): MirrorStatusResponse { + const message = createBaseMirrorStatusResponse(); + message.flowJobName = object.flowJobName ?? ""; + message.qrepStatus = (object.qrepStatus !== undefined && object.qrepStatus !== null) + ? QRepMirrorStatus.fromPartial(object.qrepStatus) + : undefined; + message.cdcStatus = (object.cdcStatus !== undefined && object.cdcStatus !== null) + ? CDCMirrorStatus.fromPartial(object.cdcStatus) + : undefined; + message.errorMessage = object.errorMessage ?? ""; + return message; + }, +}; + export type FlowServiceService = typeof FlowServiceService; export const FlowServiceService = { listPeers: { @@ -1000,6 +1665,15 @@ export const FlowServiceService = { responseSerialize: (value: ShutdownResponse) => Buffer.from(ShutdownResponse.encode(value).finish()), responseDeserialize: (value: Buffer) => ShutdownResponse.decode(value), }, + mirrorStatus: { + path: "/peerdb_route.FlowService/MirrorStatus", + requestStream: false, + responseStream: false, + requestSerialize: (value: MirrorStatusRequest) => Buffer.from(MirrorStatusRequest.encode(value).finish()), + requestDeserialize: (value: Buffer) => MirrorStatusRequest.decode(value), + responseSerialize: (value: MirrorStatusResponse) => Buffer.from(MirrorStatusResponse.encode(value).finish()), + responseDeserialize: (value: Buffer) => MirrorStatusResponse.decode(value), + }, } as const; export interface FlowServiceServer extends UntypedServiceImplementation { @@ -1009,6 +1683,7 @@ export interface FlowServiceServer extends UntypedServiceImplementation { createCdcFlow: handleUnaryCall; createQRepFlow: handleUnaryCall; shutdownFlow: handleUnaryCall; + mirrorStatus: handleUnaryCall; } export interface FlowServiceClient extends Client { @@ -1102,6 +1777,21 @@ export interface FlowServiceClient extends Client { options: Partial, callback: (error: ServiceError | null, response: ShutdownResponse) => void, ): ClientUnaryCall; + mirrorStatus( + request: MirrorStatusRequest, + callback: (error: ServiceError | null, response: MirrorStatusResponse) => void, + ): ClientUnaryCall; + mirrorStatus( + request: MirrorStatusRequest, + metadata: Metadata, + callback: (error: ServiceError | null, response: MirrorStatusResponse) => void, + ): ClientUnaryCall; + mirrorStatus( + request: MirrorStatusRequest, + metadata: Metadata, + options: Partial, + callback: (error: ServiceError | null, response: MirrorStatusResponse) => void, + ): ClientUnaryCall; } export const FlowServiceClient = makeGenericClientConstructor( @@ -1112,6 +1802,25 @@ export const FlowServiceClient = makeGenericClientConstructor( service: typeof FlowServiceService; }; +declare const self: any | undefined; +declare const window: any | undefined; +declare const global: any | undefined; +const tsProtoGlobalThis: any = (() => { + if (typeof globalThis !== "undefined") { + return globalThis; + } + if (typeof self !== "undefined") { + return self; + } + if (typeof window !== "undefined") { + return window; + } + if (typeof global !== "undefined") { + return global; + } + throw "Unable to locate global object"; +})(); + type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; export type DeepPartial = T extends Builtin ? T @@ -1123,6 +1832,40 @@ type KeysOfUnion = T extends T ? keyof T : never; export type Exact = P extends Builtin ? P : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; +function toTimestamp(date: Date): Timestamp { + const seconds = date.getTime() / 1_000; + const nanos = (date.getTime() % 1_000) * 1_000_000; + return { seconds, nanos }; +} + +function fromTimestamp(t: Timestamp): Date { + let millis = (t.seconds || 0) * 1_000; + millis += (t.nanos || 0) / 1_000_000; + return new Date(millis); +} + +function fromJsonTimestamp(o: any): Date { + if (o instanceof Date) { + return o; + } else if (typeof o === "string") { + return new Date(o); + } else { + return fromTimestamp(Timestamp.fromJSON(o)); + } +} + +function longToNumber(long: Long): number { + if (long.gt(Number.MAX_SAFE_INTEGER)) { + throw new tsProtoGlobalThis.Error("Value is larger than Number.MAX_SAFE_INTEGER"); + } + return long.toNumber(); +} + +if (_m0.util.Long !== Long) { + _m0.util.Long = Long as any; + _m0.configure(); +} + function isSet(value: any): boolean { return value !== null && value !== undefined; } From d2f0d26c61900d1323692ae0358677db53c3c8d0 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 12 Oct 2023 15:17:06 -0400 Subject: [PATCH 2/2] disable clippy for pt --- nexus/pt/src/lib.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nexus/pt/src/lib.rs b/nexus/pt/src/lib.rs index eea59cfe72..a12e67d670 100644 --- a/nexus/pt/src/lib.rs +++ b/nexus/pt/src/lib.rs @@ -1,3 +1,5 @@ +#![allow(clippy::all)] + use peerdb_peers::DbType; use sqlparser::ast::PeerType;