From b9fd5177cef9fcd12082d0e127c40a20726ada65 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 16:30:35 +0530 Subject: [PATCH 1/8] heartbeat for pg peers, create search attribute v0 --- docker-compose-dev.yml | 1 + flow/activities/flowable.go | 45 +- flow/cmd/api.go | 51 + flow/cmd/handler.go | 3 + flow/cmd/peer_data.go | 25 + flow/cmd/worker.go | 1 + flow/generated/protos/peers.pb.go | 118 +- flow/workflows/cdc_flow.go | 14 +- flow/workflows/heartbeat_flow.go | 24 + flow/workflows/qrep_flow.go | 3 + nexus/pt/src/peerdb_peers.rs | 6 + nexus/pt/src/peerdb_peers.serde.rs | 95 ++ protos/peers.proto | 4 + .../google/protobuf/descriptor.ts | 1359 +---------------- .../google/protobuf/timestamp.ts | 2 +- ui/grpc_generated/peers.ts | 63 + 16 files changed, 450 insertions(+), 1364 deletions(-) create mode 100644 flow/workflows/heartbeat_flow.go diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index 840bd556f..4c8297d18 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -63,6 +63,7 @@ services: - POSTGRES_SEEDS=catalog - DYNAMIC_CONFIG_FILE_PATH=config/dynamicconfig/development-sql.yaml image: temporalio/auto-setup:1.22 + command: sh -c 'temporal operator search-attribute create --name MirrorName --type Text' ports: - 7233:7233 volumes: diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a4b26c277..46e1d3cda 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -17,6 +17,7 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" + "github.com/jackc/pgx/v5/pgxpool" log "github.com/sirupsen/logrus" "go.temporal.io/sdk/activity" "golang.org/x/sync/errgroup" @@ -659,31 +660,37 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown return nil } -func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context, config *protos.FlowConnectionConfigs) error { - srcConn, err := connectors.GetCDCPullConnector(ctx, config.Source) - if err != nil { - return fmt.Errorf("failed to get destination connector: %w", err) - } - defer connectors.CloseConnector(srcConn) - log.WithFields(log.Fields{"flowName": config.FlowJobName}).Info("sending walheartbeat every 10 minutes") - ticker := time.NewTicker(10 * time.Minute) +func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context, configs []*protos.PostgresConfig) error { + log.Info("sending walheartbeat every 10 minutes") + ticker := time.NewTicker(10 * time.Second) for { select { case <-ctx.Done(): - log.WithFields( - log.Fields{ - "flowName": config.FlowJobName, - }).Info("context is done, exiting wal heartbeat send loop") + log.Info("context is done, exiting wal heartbeat send loop") return nil case <-ticker.C: - err = srcConn.SendWALHeartbeat() - if err != nil { - return fmt.Errorf("failed to send WAL heartbeat: %w", err) + command := ` + BEGIN; + DROP aggregate IF EXISTS PEERDB_EPHEMERAL_HEARTBEAT(float4); + CREATE AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4) (SFUNC = float4pl, STYPE = float4); + DROP aggregate PEERDB_EPHEMERAL_HEARTBEAT(float4); + END; + ` + // run above command for each Postgres peer + for _, pgConfig := range configs { + peerPool, poolErr := pgxpool.New(ctx, utils.GetPGConnectionString(pgConfig)) + if poolErr != nil { + return fmt.Errorf("error creating pool for postgres peer with host %v: %w", pgConfig.Host, poolErr) + } + + _, err := peerPool.Exec(ctx, command) + if err == nil { + log.Infof("sent wal heartbeat to postgres peer with host %v and port %v", pgConfig.Host, pgConfig.Port) + } else { + log.Warnf("warning: could not send walheartbeat to host %v: %v", pgConfig.Host, err) + } } - log.WithFields( - log.Fields{ - "flowName": config.FlowJobName, - }).Info("sent wal heartbeat") + } } } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 98bb798a6..3e2796c81 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -10,12 +10,16 @@ import ( utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/google/uuid" "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" log "github.com/sirupsen/logrus" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/reflection" + "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/client" "google.golang.org/grpc/health" "google.golang.org/grpc/health/grpc_health_v1" @@ -58,6 +62,28 @@ func setupGRPCGatewayServer(args *APIServerParams) (*http.Server, error) { return server, nil } +func KillExistingHeartbeatFlows(ctx context.Context, tc client.Client, namespace string) error { + listRes, err := tc.ListWorkflow(ctx, + &workflowservice.ListWorkflowExecutionsRequest{ + Namespace: namespace, + Query: "WorkflowType = 'HeartbeatFlowWorkflow'", + }) + if err != nil { + return fmt.Errorf("unable to list workflows: %w", err) + } + log.Info("Terminating pre-existing heartbeat flows") + for _, workflow := range listRes.Executions { + log.Info("Terminating workflow: ", workflow.Execution.WorkflowId) + err := tc.TerminateWorkflow(ctx, + workflow.Execution.WorkflowId, workflow.Execution.RunId, + "Terminate pre-existing heartbeat flows before kicking off the current one") + if err != nil && err.Error() != "workflow execution already completed" { + return fmt.Errorf("unable to terminate workflow: %w", err) + } + } + return nil +} + func APIMain(args *APIServerParams) error { ctx := args.ctx clientOptions := client.Options{ @@ -91,6 +117,31 @@ func APIMain(args *APIServerParams) error { flowHandler := NewFlowRequestHandler(tc, catalogConn) defer flowHandler.Close() + err = KillExistingHeartbeatFlows(ctx, tc, args.TemporalNamespace) + if err != nil { + return fmt.Errorf("unable to kill existing heartbeat flows: %w", err) + } + + workflowID := fmt.Sprintf("heartbeatflow-%s", uuid.New()) + workflowOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: shared.PeerFlowTaskQueue, + } + heartbeatRequest, err := flowHandler.GetPostgresPeerConfigs(ctx) + if err != nil { + return fmt.Errorf("unable to get postgres peer configs: %w", err) + } + + _, err = flowHandler.temporalClient.ExecuteWorkflow( + ctx, // context + workflowOptions, // workflow start options + peerflow.HeartbeatFlowWorkflow, // workflow function + heartbeatRequest, // workflow input + ) + if err != nil { + return fmt.Errorf("unable to start heartbeat workflow: %w", err) + } + protos.RegisterFlowServiceServer(grpcServer, flowHandler) grpc_health_v1.RegisterHealthServer(grpcServer, health.NewServer()) reflection.Register(grpcServer) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 13ce8115b..c02140407 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -309,6 +309,9 @@ func (h *FlowRequestHandler) ShutdownFlow( workflowOptions := client.StartWorkflowOptions{ ID: workflowID, TaskQueue: shared.PeerFlowTaskQueue, + SearchAttributes: map[string]interface{}{ + "MirrorName": req.FlowJobName, + }, } dropFlowHandle, err := h.temporalClient.ExecuteWorkflow( ctx, // context diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 14c8ba43c..7e70eed33 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -299,3 +299,28 @@ func (h *FlowRequestHandler) GetStatInfo( StatData: statInfoRows, }, nil } + +func (h *FlowRequestHandler) GetPostgresPeerConfigs(ctx context.Context) (*protos.PostgresPeerConfigs, error) { + var peerOptions sql.RawBytes + optionRows, err := h.pool.Query(ctx, "SELECT options FROM peers WHERE type=3") + if err != nil { + return nil, err + } + defer optionRows.Close() + var peerConfigs []*protos.PostgresConfig + for optionRows.Next() { + err := optionRows.Scan(&peerOptions) + if err != nil { + return nil, err + } + var pgPeerConfig protos.PostgresConfig + unmarshalErr := proto.Unmarshal(peerOptions, &pgPeerConfig) + if unmarshalErr != nil { + return nil, unmarshalErr + } + peerConfigs = append(peerConfigs, &pgPeerConfig) + } + return &protos.PostgresPeerConfigs{ + Configs: peerConfigs, + }, nil +} diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index 0a0901da9..240af3138 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -126,6 +126,7 @@ func WorkerMain(opts *WorkerOptions) error { w.RegisterWorkflow(peerflow.QRepFlowWorkflow) w.RegisterWorkflow(peerflow.QRepPartitionWorkflow) w.RegisterWorkflow(peerflow.DropFlowWorkflow) + w.RegisterWorkflow(peerflow.HeartbeatFlowWorkflow) w.RegisterActivity(&activities.FlowableActivity{ CatalogMirrorMonitor: catalogMirrorMonitor, }) diff --git a/flow/generated/protos/peers.pb.go b/flow/generated/protos/peers.pb.go index 6ac88a4c1..b50d2a7ba 100644 --- a/flow/generated/protos/peers.pb.go +++ b/flow/generated/protos/peers.pb.go @@ -1024,6 +1024,53 @@ func (*Peer_SqlserverConfig) isPeer_Config() {} func (*Peer_EventhubGroupConfig) isPeer_Config() {} +type PostgresPeerConfigs struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Configs []*PostgresConfig `protobuf:"bytes,1,rep,name=configs,proto3" json:"configs,omitempty"` +} + +func (x *PostgresPeerConfigs) Reset() { + *x = PostgresPeerConfigs{} + if protoimpl.UnsafeEnabled { + mi := &file_peers_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PostgresPeerConfigs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PostgresPeerConfigs) ProtoMessage() {} + +func (x *PostgresPeerConfigs) ProtoReflect() protoreflect.Message { + mi := &file_peers_proto_msgTypes[9] + 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 PostgresPeerConfigs.ProtoReflect.Descriptor instead. +func (*PostgresPeerConfigs) Descriptor() ([]byte, []int) { + return file_peers_proto_rawDescGZIP(), []int{9} +} + +func (x *PostgresPeerConfigs) GetConfigs() []*PostgresConfig { + if x != nil { + return x.Configs + } + return nil +} + var File_peers_proto protoreflect.FileDescriptor var file_peers_proto_rawDesc = []byte{ @@ -1215,22 +1262,27 @@ var file_peers_proto_rawDesc = []byte{ 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x13, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x2a, 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, - 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, - 0x46, 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, - 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, - 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x12, 0x06, - 0x0a, 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x53, 0x45, 0x52, - 0x56, 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, - 0x42, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x42, 0x0a, 0x50, - 0x65, 0x65, 0x72, 0x73, 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, 0x50, 0x65, 0x65, 0x72, - 0x73, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0xe2, - 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x5c, 0x47, 0x50, - 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x22, 0x4d, 0x0a, 0x13, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x50, 0x65, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x36, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x2a, + 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, 0x47, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, 0x46, + 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, 0x10, + 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, 0x12, + 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x12, 0x06, 0x0a, + 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x53, 0x45, 0x52, 0x56, + 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, + 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x42, 0x0a, 0x50, 0x65, + 0x65, 0x72, 0x73, 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, 0x50, 0x65, 0x65, 0x72, 0x73, + 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0xe2, 0x02, + 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x5c, 0x47, 0x50, 0x42, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1246,7 +1298,7 @@ func file_peers_proto_rawDescGZIP() []byte { } var file_peers_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 11) var file_peers_proto_goTypes = []interface{}{ (DBType)(0), // 0: peerdb_peers.DBType (*SnowflakeConfig)(nil), // 1: peerdb_peers.SnowflakeConfig @@ -1258,11 +1310,12 @@ var file_peers_proto_goTypes = []interface{}{ (*S3Config)(nil), // 7: peerdb_peers.S3Config (*SqlServerConfig)(nil), // 8: peerdb_peers.SqlServerConfig (*Peer)(nil), // 9: peerdb_peers.Peer - nil, // 10: peerdb_peers.EventHubGroupConfig.EventhubsEntry + (*PostgresPeerConfigs)(nil), // 10: peerdb_peers.PostgresPeerConfigs + nil, // 11: peerdb_peers.EventHubGroupConfig.EventhubsEntry } var file_peers_proto_depIdxs = []int32{ 4, // 0: peerdb_peers.EventHubConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig - 10, // 1: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry + 11, // 1: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry 4, // 2: peerdb_peers.EventHubGroupConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig 4, // 3: peerdb_peers.S3Config.metadata_db:type_name -> peerdb_peers.PostgresConfig 0, // 4: peerdb_peers.Peer.type:type_name -> peerdb_peers.DBType @@ -1274,12 +1327,13 @@ var file_peers_proto_depIdxs = []int32{ 7, // 10: peerdb_peers.Peer.s3_config:type_name -> peerdb_peers.S3Config 8, // 11: peerdb_peers.Peer.sqlserver_config:type_name -> peerdb_peers.SqlServerConfig 6, // 12: peerdb_peers.Peer.eventhub_group_config:type_name -> peerdb_peers.EventHubGroupConfig - 5, // 13: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig - 14, // [14:14] is the sub-list for method output_type - 14, // [14:14] is the sub-list for method input_type - 14, // [14:14] is the sub-list for extension type_name - 14, // [14:14] is the sub-list for extension extendee - 0, // [0:14] is the sub-list for field type_name + 4, // 13: peerdb_peers.PostgresPeerConfigs.configs:type_name -> peerdb_peers.PostgresConfig + 5, // 14: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig + 15, // [15:15] is the sub-list for method output_type + 15, // [15:15] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name } func init() { file_peers_proto_init() } @@ -1396,6 +1450,18 @@ func file_peers_proto_init() { return nil } } + file_peers_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PostgresPeerConfigs); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_peers_proto_msgTypes[0].OneofWrappers = []interface{}{} file_peers_proto_msgTypes[3].OneofWrappers = []interface{}{} @@ -1416,7 +1482,7 @@ func file_peers_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_peers_proto_rawDesc, NumEnums: 1, - NumMessages: 10, + NumMessages: 11, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 753cff380..3abdf6f5e 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -189,6 +189,10 @@ func CDCFlowWorkflowWithConfig( } } + searchAttributes := map[string]interface{}{ + "MirrorName": cfg.FlowJobName, + } + // start the SetupFlow workflow as a child workflow, and wait for it to complete // it should return the table schema for the source peer setupFlowID, err := GetChildWorkflowID(ctx, "setup-flow", cfg.FlowJobName) @@ -201,6 +205,7 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, + SearchAttributes: searchAttributes, } setupFlowCtx := workflow.WithChildOptions(ctx, childSetupFlowOpts) setupFlowFuture := workflow.ExecuteChildWorkflow(setupFlowCtx, SetupFlowWorkflow, cfg) @@ -220,7 +225,8 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - TaskQueue: shared.SnapshotFlowTaskQueue, + TaskQueue: shared.SnapshotFlowTaskQueue, + SearchAttributes: searchAttributes, } snapshotFlowCtx := workflow.WithChildOptions(ctx, childSnapshotFlowOpts) snapshotFlowFuture := workflow.ExecuteChildWorkflow(snapshotFlowCtx, SnapshotFlowWorkflow, cfg) @@ -317,6 +323,10 @@ func CDCFlowWorkflowWithConfig( return state, err } + searchAttributes := map[string]interface{}{ + "MirrorName": cfg.FlowJobName, + } + // execute the sync flow as a child workflow childSyncFlowOpts := workflow.ChildWorkflowOptions{ WorkflowID: syncFlowID, @@ -324,6 +334,7 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, + SearchAttributes: searchAttributes, } ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) syncFlowOptions.RelationMessageMapping = *state.RelationMessageMapping @@ -356,6 +367,7 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, + SearchAttributes: searchAttributes, } ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) diff --git a/flow/workflows/heartbeat_flow.go b/flow/workflows/heartbeat_flow.go new file mode 100644 index 000000000..b24a4bf35 --- /dev/null +++ b/flow/workflows/heartbeat_flow.go @@ -0,0 +1,24 @@ +package peerflow + +import ( + "time" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "go.temporal.io/sdk/workflow" +) + +// HeartbeatFlowWorkflow is the workflow that sets up heartbeat sending. +func HeartbeatFlowWorkflow(ctx workflow.Context, + config *protos.PostgresPeerConfigs) error { + + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 7 * 24 * time.Hour, + }) + + heartbeatFuture := workflow.ExecuteActivity(ctx, flowable.SendWALHeartbeat, config.Configs) + if err := heartbeatFuture.Get(ctx, nil); err != nil { + return err + } + + return nil +} diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 3b8e77a68..c8a7c1023 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -206,6 +206,9 @@ func (q *QRepFlowExecution) startChildWorkflow( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, + SearchAttributes: map[string]interface{}{ + "MirrorName": q.config.FlowJobName, + }, }) future := workflow.ExecuteChildWorkflow( diff --git a/nexus/pt/src/peerdb_peers.rs b/nexus/pt/src/peerdb_peers.rs index 017a36b8a..1bf1b9732 100644 --- a/nexus/pt/src/peerdb_peers.rs +++ b/nexus/pt/src/peerdb_peers.rs @@ -181,6 +181,12 @@ pub mod peer { EventhubGroupConfig(super::EventHubGroupConfig), } } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PostgresPeerConfigs { + #[prost(message, repeated, tag="1")] + pub configs: ::prost::alloc::vec::Vec, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum DbType { diff --git a/nexus/pt/src/peerdb_peers.serde.rs b/nexus/pt/src/peerdb_peers.serde.rs index 2cdf144f4..af6f53fa6 100644 --- a/nexus/pt/src/peerdb_peers.serde.rs +++ b/nexus/pt/src/peerdb_peers.serde.rs @@ -1306,6 +1306,101 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { deserializer.deserialize_struct("peerdb_peers.PostgresConfig", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PostgresPeerConfigs { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.configs.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_peers.PostgresPeerConfigs", len)?; + if !self.configs.is_empty() { + struct_ser.serialize_field("configs", &self.configs)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PostgresPeerConfigs { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "configs", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Configs, + __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 { + "configs" => Ok(GeneratedField::Configs), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PostgresPeerConfigs; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_peers.PostgresPeerConfigs") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut configs__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Configs => { + if configs__.is_some() { + return Err(serde::de::Error::duplicate_field("configs")); + } + configs__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(PostgresPeerConfigs { + configs: configs__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_peers.PostgresPeerConfigs", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for S3Config { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/peers.proto b/protos/peers.proto index fe343d906..5b7dea115 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -113,3 +113,7 @@ message Peer { EventHubGroupConfig eventhub_group_config = 10; } } + +message PostgresPeerConfigs { + repeated PostgresConfig configs = 1; +} \ No newline at end of file diff --git a/ui/grpc_generated/google/protobuf/descriptor.ts b/ui/grpc_generated/google/protobuf/descriptor.ts index 680c10f4d..0ebf9063a 100644 --- a/ui/grpc_generated/google/protobuf/descriptor.ts +++ b/ui/grpc_generated/google/protobuf/descriptor.ts @@ -4,98 +4,6 @@ import _m0 from "protobufjs/minimal"; export const protobufPackage = "google.protobuf"; -/** The full set of known editions. */ -export enum Edition { - /** EDITION_UNKNOWN - A placeholder for an unknown edition value. */ - EDITION_UNKNOWN = 0, - /** - * EDITION_PROTO2 - Legacy syntax "editions". These pre-date editions, but behave much like - * distinct editions. These can't be used to specify the edition of proto - * files, but feature definitions must supply proto2/proto3 defaults for - * backwards compatibility. - */ - EDITION_PROTO2 = 998, - EDITION_PROTO3 = 999, - /** - * EDITION_2023 - Editions that have been released. The specific values are arbitrary and - * should not be depended on, but they will always be time-ordered for easy - * comparison. - */ - EDITION_2023 = 1000, - /** - * EDITION_1_TEST_ONLY - Placeholder editions for testing feature resolution. These should not be - * used or relyed on outside of tests. - */ - EDITION_1_TEST_ONLY = 1, - EDITION_2_TEST_ONLY = 2, - EDITION_99997_TEST_ONLY = 99997, - EDITION_99998_TEST_ONLY = 99998, - EDITION_99999_TEST_ONLY = 99999, - UNRECOGNIZED = -1, -} - -export function editionFromJSON(object: any): Edition { - switch (object) { - case 0: - case "EDITION_UNKNOWN": - return Edition.EDITION_UNKNOWN; - case 998: - case "EDITION_PROTO2": - return Edition.EDITION_PROTO2; - case 999: - case "EDITION_PROTO3": - return Edition.EDITION_PROTO3; - case 1000: - case "EDITION_2023": - return Edition.EDITION_2023; - case 1: - case "EDITION_1_TEST_ONLY": - return Edition.EDITION_1_TEST_ONLY; - case 2: - case "EDITION_2_TEST_ONLY": - return Edition.EDITION_2_TEST_ONLY; - case 99997: - case "EDITION_99997_TEST_ONLY": - return Edition.EDITION_99997_TEST_ONLY; - case 99998: - case "EDITION_99998_TEST_ONLY": - return Edition.EDITION_99998_TEST_ONLY; - case 99999: - case "EDITION_99999_TEST_ONLY": - return Edition.EDITION_99999_TEST_ONLY; - case -1: - case "UNRECOGNIZED": - default: - return Edition.UNRECOGNIZED; - } -} - -export function editionToJSON(object: Edition): string { - switch (object) { - case Edition.EDITION_UNKNOWN: - return "EDITION_UNKNOWN"; - case Edition.EDITION_PROTO2: - return "EDITION_PROTO2"; - case Edition.EDITION_PROTO3: - return "EDITION_PROTO3"; - case Edition.EDITION_2023: - return "EDITION_2023"; - case Edition.EDITION_1_TEST_ONLY: - return "EDITION_1_TEST_ONLY"; - case Edition.EDITION_2_TEST_ONLY: - return "EDITION_2_TEST_ONLY"; - case Edition.EDITION_99997_TEST_ONLY: - return "EDITION_99997_TEST_ONLY"; - case Edition.EDITION_99998_TEST_ONLY: - return "EDITION_99998_TEST_ONLY"; - case Edition.EDITION_99999_TEST_ONLY: - return "EDITION_99999_TEST_ONLY"; - case Edition.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - /** * The protocol compiler can output a FileDescriptorSet containing the .proto * files it parses. @@ -143,8 +51,8 @@ export interface FileDescriptorProto { * If `edition` is present, this value must be "editions". */ syntax: string; - /** The edition of the proto file. */ - edition: Edition; + /** The edition of the proto file, which is an opaque string. */ + edition: string; } /** Describes a message type. */ @@ -188,84 +96,6 @@ export interface DescriptorProto_ReservedRange { export interface ExtensionRangeOptions { /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; - /** - * For external users: DO NOT USE. We are in the process of open sourcing - * extension declaration and executing internal cleanups before it can be - * used externally. - */ - declaration: ExtensionRangeOptions_Declaration[]; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; - /** - * The verification state of the range. - * TODO: flip the default to DECLARATION once all empty ranges - * are marked as UNVERIFIED. - */ - verification: ExtensionRangeOptions_VerificationState; -} - -/** The verification state of the extension range. */ -export enum ExtensionRangeOptions_VerificationState { - /** DECLARATION - All the extensions of the range must be declared. */ - DECLARATION = 0, - UNVERIFIED = 1, - UNRECOGNIZED = -1, -} - -export function extensionRangeOptions_VerificationStateFromJSON(object: any): ExtensionRangeOptions_VerificationState { - switch (object) { - case 0: - case "DECLARATION": - return ExtensionRangeOptions_VerificationState.DECLARATION; - case 1: - case "UNVERIFIED": - return ExtensionRangeOptions_VerificationState.UNVERIFIED; - case -1: - case "UNRECOGNIZED": - default: - return ExtensionRangeOptions_VerificationState.UNRECOGNIZED; - } -} - -export function extensionRangeOptions_VerificationStateToJSON(object: ExtensionRangeOptions_VerificationState): string { - switch (object) { - case ExtensionRangeOptions_VerificationState.DECLARATION: - return "DECLARATION"; - case ExtensionRangeOptions_VerificationState.UNVERIFIED: - return "UNVERIFIED"; - case ExtensionRangeOptions_VerificationState.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export interface ExtensionRangeOptions_Declaration { - /** The extension number declared within the extension range. */ - number: number; - /** - * The fully-qualified name of the extension field. There must be a leading - * dot in front of the full name. - */ - fullName: string; - /** - * The fully-qualified type name of the extension field. Unlike - * Metadata.type, Declaration.type must have a leading dot for messages - * and enums. - */ - type: string; - /** - * If true, indicates that the number is reserved in the extension range, - * and any extension field with the number will fail to compile. Set this - * when a declared extension field is deleted. - */ - reserved: boolean; - /** - * If true, indicates that the extension must be defined as repeated. - * Otherwise the extension must be defined as optional. - */ - repeated: boolean; } /** Describes a field within a message. */ @@ -363,10 +193,9 @@ export enum FieldDescriptorProto_Type { TYPE_STRING = 9, /** * TYPE_GROUP - Tag-delimited aggregate. - * Group type is deprecated and not supported after google.protobuf. However, Proto3 + * Group type is deprecated and not supported in proto3. However, Proto3 * implementations should still be able to parse the group wire format and - * treat group fields as unknown fields. In Editions, the group wire format - * can be enabled via the `message_encoding` feature. + * treat group fields as unknown fields. */ TYPE_GROUP = 10, /** TYPE_MESSAGE - Length-delimited aggregate. */ @@ -494,13 +323,8 @@ export function fieldDescriptorProto_TypeToJSON(object: FieldDescriptorProto_Typ export enum FieldDescriptorProto_Label { /** LABEL_OPTIONAL - 0 is reserved for errors */ LABEL_OPTIONAL = 1, - LABEL_REPEATED = 3, - /** - * LABEL_REQUIRED - The required label is only allowed in google.protobuf. In proto3 and Editions - * it's explicitly prohibited. In Editions, the `field_presence` feature - * can be used to get this behavior. - */ LABEL_REQUIRED = 2, + LABEL_REPEATED = 3, UNRECOGNIZED = -1, } @@ -509,12 +333,12 @@ export function fieldDescriptorProto_LabelFromJSON(object: any): FieldDescriptor case 1: case "LABEL_OPTIONAL": return FieldDescriptorProto_Label.LABEL_OPTIONAL; - case 3: - case "LABEL_REPEATED": - return FieldDescriptorProto_Label.LABEL_REPEATED; case 2: case "LABEL_REQUIRED": return FieldDescriptorProto_Label.LABEL_REQUIRED; + case 3: + case "LABEL_REPEATED": + return FieldDescriptorProto_Label.LABEL_REPEATED; case -1: case "UNRECOGNIZED": default: @@ -526,10 +350,10 @@ export function fieldDescriptorProto_LabelToJSON(object: FieldDescriptorProto_La switch (object) { case FieldDescriptorProto_Label.LABEL_OPTIONAL: return "LABEL_OPTIONAL"; - case FieldDescriptorProto_Label.LABEL_REPEATED: - return "LABEL_REPEATED"; case FieldDescriptorProto_Label.LABEL_REQUIRED: return "LABEL_REQUIRED"; + case FieldDescriptorProto_Label.LABEL_REPEATED: + return "LABEL_REPEATED"; case FieldDescriptorProto_Label.UNRECOGNIZED: default: return "UNRECOGNIZED"; @@ -723,10 +547,6 @@ export interface FileOptions { * determining the ruby package. */ rubyPackage: string; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** * The parser stores options it doesn't recognize here. * See the documentation for the "Options" section above. @@ -845,16 +665,12 @@ export interface MessageOptions { * This should only be used as a temporary measure against broken builds due * to the change in behavior for JSON field name conflicts. * - * TODO This is legacy behavior we plan to remove once downstream + * TODO(b/261750190) This is legacy behavior we plan to remove once downstream * teams have had time to migrate. * * @deprecated */ deprecatedLegacyJsonFieldConflicts: boolean; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -863,10 +679,8 @@ export interface FieldOptions { /** * The ctype option instructs the C++ code generator to use a different * representation of the field than it normally would. See the specific - * options below. This option is only implemented to support use of - * [ctype=CORD] and [ctype=STRING] (the default) on non-repeated fields of - * type "bytes" in the open source release -- sorry, we'll try to include - * other types in a future version! + * options below. This option is not yet implemented in the open source + * release -- sorry, we'll try to include it in a future version! */ ctype: FieldOptions_CType; /** @@ -874,9 +688,7 @@ export interface FieldOptions { * a more efficient representation on the wire. Rather than repeatedly * writing the tag and type for each element, the entire array is encoded as * a single length-delimited blob. In proto3, only explicit setting it to - * false will avoid using packed encoding. This option is prohibited in - * Editions, but the `repeated_field_encoding` feature can be used to control - * the behavior. + * false will avoid using packed encoding. */ packed: boolean; /** @@ -947,12 +759,7 @@ export interface FieldOptions { */ debugRedact: boolean; retention: FieldOptions_OptionRetention; - targets: FieldOptions_OptionTargetType[]; - editionDefaults: FieldOptions_EditionDefault[]; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; + target: FieldOptions_OptionTargetType; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -960,14 +767,6 @@ export interface FieldOptions { export enum FieldOptions_CType { /** STRING - Default mode. */ STRING = 0, - /** - * CORD - The option [ctype=CORD] may be applied to a non-repeated field of type - * "bytes". It indicates that in C++, the data should be stored in a Cord - * instead of a string. For very large strings, this may reduce memory - * fragmentation. It may also allow better performance when parsing from a - * Cord, or when parsing with aliasing enabled, as the parsed Cord may then - * alias the original buffer. - */ CORD = 1, STRING_PIECE = 2, UNRECOGNIZED = -1, @@ -1178,17 +977,7 @@ export function fieldOptions_OptionTargetTypeToJSON(object: FieldOptions_OptionT } } -export interface FieldOptions_EditionDefault { - edition: Edition; - /** Textproto value. */ - value: string; -} - export interface OneofOptions { - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1211,16 +1000,12 @@ export interface EnumOptions { * and strips underscored from the fields before comparison in proto3 only. * The new behavior takes `json_name` into account and applies to proto2 as * well. - * TODO Remove this legacy behavior once downstream teams have + * TODO(b/261750190) Remove this legacy behavior once downstream teams have * had time to migrate. * * @deprecated */ deprecatedLegacyJsonFieldConflicts: boolean; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1233,25 +1018,11 @@ export interface EnumValueOptions { * this is a formalization for deprecating enum values. */ deprecated: boolean; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; - /** - * Indicate that fields annotated with this enum value should not be printed - * out when using debug formats, e.g. when the field contains sensitive - * credentials. - */ - debugRedact: boolean; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } export interface ServiceOptions { - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** * Is this service deprecated? * Depending on the target platform, this can emit Deprecated annotations @@ -1272,10 +1043,6 @@ export interface MethodOptions { */ deprecated: boolean; idempotencyLevel: MethodOptions_IdempotencyLevel; - /** Any features defined in the specific edition. */ - features: - | FeatureSet - | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1360,294 +1127,6 @@ export interface UninterpretedOption_NamePart { isExtension: boolean; } -/** - * TODO Enums in C++ gencode (and potentially other languages) are - * not well scoped. This means that each of the feature enums below can clash - * with each other. The short names we've chosen maximize call-site - * readability, but leave us very open to this scenario. A future feature will - * be designed and implemented to handle this, hopefully before we ever hit a - * conflict here. - */ -export interface FeatureSet { - fieldPresence: FeatureSet_FieldPresence; - enumType: FeatureSet_EnumType; - repeatedFieldEncoding: FeatureSet_RepeatedFieldEncoding; - utf8Validation: FeatureSet_Utf8Validation; - messageEncoding: FeatureSet_MessageEncoding; - jsonFormat: FeatureSet_JsonFormat; -} - -export enum FeatureSet_FieldPresence { - FIELD_PRESENCE_UNKNOWN = 0, - EXPLICIT = 1, - IMPLICIT = 2, - LEGACY_REQUIRED = 3, - UNRECOGNIZED = -1, -} - -export function featureSet_FieldPresenceFromJSON(object: any): FeatureSet_FieldPresence { - switch (object) { - case 0: - case "FIELD_PRESENCE_UNKNOWN": - return FeatureSet_FieldPresence.FIELD_PRESENCE_UNKNOWN; - case 1: - case "EXPLICIT": - return FeatureSet_FieldPresence.EXPLICIT; - case 2: - case "IMPLICIT": - return FeatureSet_FieldPresence.IMPLICIT; - case 3: - case "LEGACY_REQUIRED": - return FeatureSet_FieldPresence.LEGACY_REQUIRED; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_FieldPresence.UNRECOGNIZED; - } -} - -export function featureSet_FieldPresenceToJSON(object: FeatureSet_FieldPresence): string { - switch (object) { - case FeatureSet_FieldPresence.FIELD_PRESENCE_UNKNOWN: - return "FIELD_PRESENCE_UNKNOWN"; - case FeatureSet_FieldPresence.EXPLICIT: - return "EXPLICIT"; - case FeatureSet_FieldPresence.IMPLICIT: - return "IMPLICIT"; - case FeatureSet_FieldPresence.LEGACY_REQUIRED: - return "LEGACY_REQUIRED"; - case FeatureSet_FieldPresence.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export enum FeatureSet_EnumType { - ENUM_TYPE_UNKNOWN = 0, - OPEN = 1, - CLOSED = 2, - UNRECOGNIZED = -1, -} - -export function featureSet_EnumTypeFromJSON(object: any): FeatureSet_EnumType { - switch (object) { - case 0: - case "ENUM_TYPE_UNKNOWN": - return FeatureSet_EnumType.ENUM_TYPE_UNKNOWN; - case 1: - case "OPEN": - return FeatureSet_EnumType.OPEN; - case 2: - case "CLOSED": - return FeatureSet_EnumType.CLOSED; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_EnumType.UNRECOGNIZED; - } -} - -export function featureSet_EnumTypeToJSON(object: FeatureSet_EnumType): string { - switch (object) { - case FeatureSet_EnumType.ENUM_TYPE_UNKNOWN: - return "ENUM_TYPE_UNKNOWN"; - case FeatureSet_EnumType.OPEN: - return "OPEN"; - case FeatureSet_EnumType.CLOSED: - return "CLOSED"; - case FeatureSet_EnumType.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export enum FeatureSet_RepeatedFieldEncoding { - REPEATED_FIELD_ENCODING_UNKNOWN = 0, - PACKED = 1, - EXPANDED = 2, - UNRECOGNIZED = -1, -} - -export function featureSet_RepeatedFieldEncodingFromJSON(object: any): FeatureSet_RepeatedFieldEncoding { - switch (object) { - case 0: - case "REPEATED_FIELD_ENCODING_UNKNOWN": - return FeatureSet_RepeatedFieldEncoding.REPEATED_FIELD_ENCODING_UNKNOWN; - case 1: - case "PACKED": - return FeatureSet_RepeatedFieldEncoding.PACKED; - case 2: - case "EXPANDED": - return FeatureSet_RepeatedFieldEncoding.EXPANDED; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_RepeatedFieldEncoding.UNRECOGNIZED; - } -} - -export function featureSet_RepeatedFieldEncodingToJSON(object: FeatureSet_RepeatedFieldEncoding): string { - switch (object) { - case FeatureSet_RepeatedFieldEncoding.REPEATED_FIELD_ENCODING_UNKNOWN: - return "REPEATED_FIELD_ENCODING_UNKNOWN"; - case FeatureSet_RepeatedFieldEncoding.PACKED: - return "PACKED"; - case FeatureSet_RepeatedFieldEncoding.EXPANDED: - return "EXPANDED"; - case FeatureSet_RepeatedFieldEncoding.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export enum FeatureSet_Utf8Validation { - UTF8_VALIDATION_UNKNOWN = 0, - NONE = 1, - VERIFY = 2, - UNRECOGNIZED = -1, -} - -export function featureSet_Utf8ValidationFromJSON(object: any): FeatureSet_Utf8Validation { - switch (object) { - case 0: - case "UTF8_VALIDATION_UNKNOWN": - return FeatureSet_Utf8Validation.UTF8_VALIDATION_UNKNOWN; - case 1: - case "NONE": - return FeatureSet_Utf8Validation.NONE; - case 2: - case "VERIFY": - return FeatureSet_Utf8Validation.VERIFY; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_Utf8Validation.UNRECOGNIZED; - } -} - -export function featureSet_Utf8ValidationToJSON(object: FeatureSet_Utf8Validation): string { - switch (object) { - case FeatureSet_Utf8Validation.UTF8_VALIDATION_UNKNOWN: - return "UTF8_VALIDATION_UNKNOWN"; - case FeatureSet_Utf8Validation.NONE: - return "NONE"; - case FeatureSet_Utf8Validation.VERIFY: - return "VERIFY"; - case FeatureSet_Utf8Validation.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export enum FeatureSet_MessageEncoding { - MESSAGE_ENCODING_UNKNOWN = 0, - LENGTH_PREFIXED = 1, - DELIMITED = 2, - UNRECOGNIZED = -1, -} - -export function featureSet_MessageEncodingFromJSON(object: any): FeatureSet_MessageEncoding { - switch (object) { - case 0: - case "MESSAGE_ENCODING_UNKNOWN": - return FeatureSet_MessageEncoding.MESSAGE_ENCODING_UNKNOWN; - case 1: - case "LENGTH_PREFIXED": - return FeatureSet_MessageEncoding.LENGTH_PREFIXED; - case 2: - case "DELIMITED": - return FeatureSet_MessageEncoding.DELIMITED; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_MessageEncoding.UNRECOGNIZED; - } -} - -export function featureSet_MessageEncodingToJSON(object: FeatureSet_MessageEncoding): string { - switch (object) { - case FeatureSet_MessageEncoding.MESSAGE_ENCODING_UNKNOWN: - return "MESSAGE_ENCODING_UNKNOWN"; - case FeatureSet_MessageEncoding.LENGTH_PREFIXED: - return "LENGTH_PREFIXED"; - case FeatureSet_MessageEncoding.DELIMITED: - return "DELIMITED"; - case FeatureSet_MessageEncoding.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -export enum FeatureSet_JsonFormat { - JSON_FORMAT_UNKNOWN = 0, - ALLOW = 1, - LEGACY_BEST_EFFORT = 2, - UNRECOGNIZED = -1, -} - -export function featureSet_JsonFormatFromJSON(object: any): FeatureSet_JsonFormat { - switch (object) { - case 0: - case "JSON_FORMAT_UNKNOWN": - return FeatureSet_JsonFormat.JSON_FORMAT_UNKNOWN; - case 1: - case "ALLOW": - return FeatureSet_JsonFormat.ALLOW; - case 2: - case "LEGACY_BEST_EFFORT": - return FeatureSet_JsonFormat.LEGACY_BEST_EFFORT; - case -1: - case "UNRECOGNIZED": - default: - return FeatureSet_JsonFormat.UNRECOGNIZED; - } -} - -export function featureSet_JsonFormatToJSON(object: FeatureSet_JsonFormat): string { - switch (object) { - case FeatureSet_JsonFormat.JSON_FORMAT_UNKNOWN: - return "JSON_FORMAT_UNKNOWN"; - case FeatureSet_JsonFormat.ALLOW: - return "ALLOW"; - case FeatureSet_JsonFormat.LEGACY_BEST_EFFORT: - return "LEGACY_BEST_EFFORT"; - case FeatureSet_JsonFormat.UNRECOGNIZED: - default: - return "UNRECOGNIZED"; - } -} - -/** - * A compiled specification for the defaults of a set of features. These - * messages are generated from FeatureSet extensions and can be used to seed - * feature resolution. The resolution with this object becomes a simple search - * for the closest matching edition, followed by proto merges. - */ -export interface FeatureSetDefaults { - defaults: FeatureSetDefaults_FeatureSetEditionDefault[]; - /** - * The minimum supported edition (inclusive) when this was constructed. - * Editions before this will not have defaults. - */ - minimumEdition: Edition; - /** - * The maximum known edition (inclusive) when this was constructed. Editions - * after this will not have reliable defaults. - */ - maximumEdition: Edition; -} - -/** - * A map from every known edition with a unique set of defaults to its - * defaults. Not all editions may be contained here. For a given edition, - * the defaults at the closest matching edition ordered at or before it should - * be used. This field must be in strict ascending order by edition. - */ -export interface FeatureSetDefaults_FeatureSetEditionDefault { - edition: Edition; - features: FeatureSet | undefined; -} - /** * Encapsulates information about the original source file from which a * FileDescriptorProto was generated. @@ -1942,7 +1421,7 @@ function createBaseFileDescriptorProto(): FileDescriptorProto { options: undefined, sourceCodeInfo: undefined, syntax: "", - edition: 0, + edition: "", }; } @@ -1988,8 +1467,8 @@ export const FileDescriptorProto = { if (message.syntax !== "") { writer.uint32(98).string(message.syntax); } - if (message.edition !== 0) { - writer.uint32(112).int32(message.edition); + if (message.edition !== "") { + writer.uint32(106).string(message.edition); } return writer; }, @@ -2105,12 +1584,12 @@ export const FileDescriptorProto = { message.syntax = reader.string(); continue; - case 14: - if (tag !== 112) { + case 13: + if (tag !== 106) { break; } - message.edition = reader.int32() as any; + message.edition = reader.string(); continue; } if ((tag & 7) === 4 || tag === 0) { @@ -2141,7 +1620,7 @@ export const FileDescriptorProto = { options: isSet(object.options) ? FileOptions.fromJSON(object.options) : undefined, sourceCodeInfo: isSet(object.sourceCodeInfo) ? SourceCodeInfo.fromJSON(object.sourceCodeInfo) : undefined, syntax: isSet(object.syntax) ? String(object.syntax) : "", - edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, + edition: isSet(object.edition) ? String(object.edition) : "", }; }, @@ -2183,8 +1662,8 @@ export const FileDescriptorProto = { if (message.syntax !== "") { obj.syntax = message.syntax; } - if (message.edition !== 0) { - obj.edition = editionToJSON(message.edition); + if (message.edition !== "") { + obj.edition = message.edition; } return obj; }, @@ -2210,7 +1689,7 @@ export const FileDescriptorProto = { ? SourceCodeInfo.fromPartial(object.sourceCodeInfo) : undefined; message.syntax = object.syntax ?? ""; - message.edition = object.edition ?? 0; + message.edition = object.edition ?? ""; return message; }, }; @@ -2599,7 +2078,7 @@ export const DescriptorProto_ReservedRange = { }; function createBaseExtensionRangeOptions(): ExtensionRangeOptions { - return { uninterpretedOption: [], declaration: [], features: undefined, verification: 0 }; + return { uninterpretedOption: [] }; } export const ExtensionRangeOptions = { @@ -2607,15 +2086,6 @@ export const ExtensionRangeOptions = { for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } - for (const v of message.declaration) { - ExtensionRangeOptions_Declaration.encode(v!, writer.uint32(18).fork()).ldelim(); - } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(402).fork()).ldelim(); - } - if (message.verification !== 0) { - writer.uint32(24).int32(message.verification); - } return writer; }, @@ -2633,27 +2103,6 @@ export const ExtensionRangeOptions = { message.uninterpretedOption.push(UninterpretedOption.decode(reader, reader.uint32())); continue; - case 2: - if (tag !== 18) { - break; - } - - message.declaration.push(ExtensionRangeOptions_Declaration.decode(reader, reader.uint32())); - continue; - case 50: - if (tag !== 402) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; - case 3: - if (tag !== 24) { - break; - } - - message.verification = reader.int32() as any; - continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -2668,13 +2117,6 @@ export const ExtensionRangeOptions = { uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], - declaration: Array.isArray(object?.declaration) - ? object.declaration.map((e: any) => ExtensionRangeOptions_Declaration.fromJSON(e)) - : [], - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, - verification: isSet(object.verification) - ? extensionRangeOptions_VerificationStateFromJSON(object.verification) - : 0, }; }, @@ -2683,15 +2125,6 @@ export const ExtensionRangeOptions = { if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } - if (message.declaration?.length) { - obj.declaration = message.declaration.map((e) => ExtensionRangeOptions_Declaration.toJSON(e)); - } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } - if (message.verification !== 0) { - obj.verification = extensionRangeOptions_VerificationStateToJSON(message.verification); - } return obj; }, @@ -2701,134 +2134,6 @@ export const ExtensionRangeOptions = { fromPartial, I>>(object: I): ExtensionRangeOptions { const message = createBaseExtensionRangeOptions(); message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; - message.declaration = object.declaration?.map((e) => ExtensionRangeOptions_Declaration.fromPartial(e)) || []; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; - message.verification = object.verification ?? 0; - return message; - }, -}; - -function createBaseExtensionRangeOptions_Declaration(): ExtensionRangeOptions_Declaration { - return { number: 0, fullName: "", type: "", reserved: false, repeated: false }; -} - -export const ExtensionRangeOptions_Declaration = { - encode(message: ExtensionRangeOptions_Declaration, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.number !== 0) { - writer.uint32(8).int32(message.number); - } - if (message.fullName !== "") { - writer.uint32(18).string(message.fullName); - } - if (message.type !== "") { - writer.uint32(26).string(message.type); - } - if (message.reserved === true) { - writer.uint32(40).bool(message.reserved); - } - if (message.repeated === true) { - writer.uint32(48).bool(message.repeated); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): ExtensionRangeOptions_Declaration { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseExtensionRangeOptions_Declaration(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 8) { - break; - } - - message.number = reader.int32(); - continue; - case 2: - if (tag !== 18) { - break; - } - - message.fullName = reader.string(); - continue; - case 3: - if (tag !== 26) { - break; - } - - message.type = reader.string(); - continue; - case 5: - if (tag !== 40) { - break; - } - - message.reserved = reader.bool(); - continue; - case 6: - if (tag !== 48) { - break; - } - - message.repeated = reader.bool(); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): ExtensionRangeOptions_Declaration { - return { - number: isSet(object.number) ? Number(object.number) : 0, - fullName: isSet(object.fullName) ? String(object.fullName) : "", - type: isSet(object.type) ? String(object.type) : "", - reserved: isSet(object.reserved) ? Boolean(object.reserved) : false, - repeated: isSet(object.repeated) ? Boolean(object.repeated) : false, - }; - }, - - toJSON(message: ExtensionRangeOptions_Declaration): unknown { - const obj: any = {}; - if (message.number !== 0) { - obj.number = Math.round(message.number); - } - if (message.fullName !== "") { - obj.fullName = message.fullName; - } - if (message.type !== "") { - obj.type = message.type; - } - if (message.reserved === true) { - obj.reserved = message.reserved; - } - if (message.repeated === true) { - obj.repeated = message.repeated; - } - return obj; - }, - - create, I>>( - base?: I, - ): ExtensionRangeOptions_Declaration { - return ExtensionRangeOptions_Declaration.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>( - object: I, - ): ExtensionRangeOptions_Declaration { - const message = createBaseExtensionRangeOptions_Declaration(); - message.number = object.number ?? 0; - message.fullName = object.fullName ?? ""; - message.type = object.type ?? ""; - message.reserved = object.reserved ?? false; - message.repeated = object.repeated ?? false; return message; }, }; @@ -3678,7 +2983,6 @@ function createBaseFileOptions(): FileOptions { phpNamespace: "", phpMetadataNamespace: "", rubyPackage: "", - features: undefined, uninterpretedOption: [], }; } @@ -3745,9 +3049,6 @@ export const FileOptions = { if (message.rubyPackage !== "") { writer.uint32(362).string(message.rubyPackage); } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(402).fork()).ldelim(); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3901,13 +3202,6 @@ export const FileOptions = { message.rubyPackage = reader.string(); continue; - case 50: - if (tag !== 402) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 999: if (tag !== 7994) { break; @@ -3948,7 +3242,6 @@ export const FileOptions = { phpNamespace: isSet(object.phpNamespace) ? String(object.phpNamespace) : "", phpMetadataNamespace: isSet(object.phpMetadataNamespace) ? String(object.phpMetadataNamespace) : "", rubyPackage: isSet(object.rubyPackage) ? String(object.rubyPackage) : "", - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4017,9 +3310,6 @@ export const FileOptions = { if (message.rubyPackage !== "") { obj.rubyPackage = message.rubyPackage; } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4051,9 +3341,6 @@ export const FileOptions = { message.phpNamespace = object.phpNamespace ?? ""; message.phpMetadataNamespace = object.phpMetadataNamespace ?? ""; message.rubyPackage = object.rubyPackage ?? ""; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -4066,7 +3353,6 @@ function createBaseMessageOptions(): MessageOptions { deprecated: false, mapEntry: false, deprecatedLegacyJsonFieldConflicts: false, - features: undefined, uninterpretedOption: [], }; } @@ -4088,9 +3374,6 @@ export const MessageOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { writer.uint32(88).bool(message.deprecatedLegacyJsonFieldConflicts); } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(98).fork()).ldelim(); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -4139,13 +3422,6 @@ export const MessageOptions = { message.deprecatedLegacyJsonFieldConflicts = reader.bool(); continue; - case 12: - if (tag !== 98) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 999: if (tag !== 7994) { break; @@ -4173,7 +3449,6 @@ export const MessageOptions = { deprecatedLegacyJsonFieldConflicts: isSet(object.deprecatedLegacyJsonFieldConflicts) ? Boolean(object.deprecatedLegacyJsonFieldConflicts) : false, - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4197,9 +3472,6 @@ export const MessageOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4216,9 +3488,6 @@ export const MessageOptions = { message.deprecated = object.deprecated ?? false; message.mapEntry = object.mapEntry ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -4235,9 +3504,7 @@ function createBaseFieldOptions(): FieldOptions { weak: false, debugRedact: false, retention: 0, - targets: [], - editionDefaults: [], - features: undefined, + target: 0, uninterpretedOption: [], }; } @@ -4271,16 +3538,8 @@ export const FieldOptions = { if (message.retention !== 0) { writer.uint32(136).int32(message.retention); } - writer.uint32(154).fork(); - for (const v of message.targets) { - writer.int32(v); - } - writer.ldelim(); - for (const v of message.editionDefaults) { - FieldOptions_EditionDefault.encode(v!, writer.uint32(162).fork()).ldelim(); - } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(170).fork()).ldelim(); + if (message.target !== 0) { + writer.uint32(144).int32(message.target); } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); @@ -4358,36 +3617,12 @@ export const FieldOptions = { message.retention = reader.int32() as any; continue; - case 19: - if (tag === 152) { - message.targets.push(reader.int32() as any); - - continue; - } - - if (tag === 154) { - const end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) { - message.targets.push(reader.int32() as any); - } - - continue; - } - - break; - case 20: - if (tag !== 162) { - break; - } - - message.editionDefaults.push(FieldOptions_EditionDefault.decode(reader, reader.uint32())); - continue; - case 21: - if (tag !== 170) { + case 18: + if (tag !== 144) { break; } - message.features = FeatureSet.decode(reader, reader.uint32()); + message.target = reader.int32() as any; continue; case 999: if (tag !== 7994) { @@ -4416,13 +3651,7 @@ export const FieldOptions = { weak: isSet(object.weak) ? Boolean(object.weak) : false, debugRedact: isSet(object.debugRedact) ? Boolean(object.debugRedact) : false, retention: isSet(object.retention) ? fieldOptions_OptionRetentionFromJSON(object.retention) : 0, - targets: Array.isArray(object?.targets) - ? object.targets.map((e: any) => fieldOptions_OptionTargetTypeFromJSON(e)) - : [], - editionDefaults: Array.isArray(object?.editionDefaults) - ? object.editionDefaults.map((e: any) => FieldOptions_EditionDefault.fromJSON(e)) - : [], - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, + target: isSet(object.target) ? fieldOptions_OptionTargetTypeFromJSON(object.target) : 0, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4458,14 +3687,8 @@ export const FieldOptions = { if (message.retention !== 0) { obj.retention = fieldOptions_OptionRetentionToJSON(message.retention); } - if (message.targets?.length) { - obj.targets = message.targets.map((e) => fieldOptions_OptionTargetTypeToJSON(e)); - } - if (message.editionDefaults?.length) { - obj.editionDefaults = message.editionDefaults.map((e) => FieldOptions_EditionDefault.toJSON(e)); - } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); + if (message.target !== 0) { + obj.target = fieldOptions_OptionTargetTypeToJSON(message.target); } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); @@ -4487,99 +3710,18 @@ export const FieldOptions = { message.weak = object.weak ?? false; message.debugRedact = object.debugRedact ?? false; message.retention = object.retention ?? 0; - message.targets = object.targets?.map((e) => e) || []; - message.editionDefaults = object.editionDefaults?.map((e) => FieldOptions_EditionDefault.fromPartial(e)) || []; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; + message.target = object.target ?? 0; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; -function createBaseFieldOptions_EditionDefault(): FieldOptions_EditionDefault { - return { edition: 0, value: "" }; -} - -export const FieldOptions_EditionDefault = { - encode(message: FieldOptions_EditionDefault, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.edition !== 0) { - writer.uint32(24).int32(message.edition); - } - if (message.value !== "") { - writer.uint32(18).string(message.value); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): FieldOptions_EditionDefault { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseFieldOptions_EditionDefault(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 3: - if (tag !== 24) { - break; - } - - message.edition = reader.int32() as any; - continue; - case 2: - if (tag !== 18) { - break; - } - - message.value = reader.string(); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): FieldOptions_EditionDefault { - return { - edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, - value: isSet(object.value) ? String(object.value) : "", - }; - }, - - toJSON(message: FieldOptions_EditionDefault): unknown { - const obj: any = {}; - if (message.edition !== 0) { - obj.edition = editionToJSON(message.edition); - } - if (message.value !== "") { - obj.value = message.value; - } - return obj; - }, - - create, I>>(base?: I): FieldOptions_EditionDefault { - return FieldOptions_EditionDefault.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): FieldOptions_EditionDefault { - const message = createBaseFieldOptions_EditionDefault(); - message.edition = object.edition ?? 0; - message.value = object.value ?? ""; - return message; - }, -}; - function createBaseOneofOptions(): OneofOptions { - return { features: undefined, uninterpretedOption: [] }; + return { uninterpretedOption: [] }; } export const OneofOptions = { encode(message: OneofOptions, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(10).fork()).ldelim(); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -4593,13 +3735,6 @@ export const OneofOptions = { while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { - case 1: - if (tag !== 10) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 999: if (tag !== 7994) { break; @@ -4618,7 +3753,6 @@ export const OneofOptions = { fromJSON(object: any): OneofOptions { return { - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4627,9 +3761,6 @@ export const OneofOptions = { toJSON(message: OneofOptions): unknown { const obj: any = {}; - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4641,22 +3772,13 @@ export const OneofOptions = { }, fromPartial, I>>(object: I): OneofOptions { const message = createBaseOneofOptions(); - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseEnumOptions(): EnumOptions { - return { - allowAlias: false, - deprecated: false, - deprecatedLegacyJsonFieldConflicts: false, - features: undefined, - uninterpretedOption: [], - }; + return { allowAlias: false, deprecated: false, deprecatedLegacyJsonFieldConflicts: false, uninterpretedOption: [] }; } export const EnumOptions = { @@ -4670,9 +3792,6 @@ export const EnumOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { writer.uint32(48).bool(message.deprecatedLegacyJsonFieldConflicts); } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(58).fork()).ldelim(); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -4707,13 +3826,6 @@ export const EnumOptions = { message.deprecatedLegacyJsonFieldConflicts = reader.bool(); continue; - case 7: - if (tag !== 58) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 999: if (tag !== 7994) { break; @@ -4737,7 +3849,6 @@ export const EnumOptions = { deprecatedLegacyJsonFieldConflicts: isSet(object.deprecatedLegacyJsonFieldConflicts) ? Boolean(object.deprecatedLegacyJsonFieldConflicts) : false, - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4755,9 +3866,6 @@ export const EnumOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4772,16 +3880,13 @@ export const EnumOptions = { message.allowAlias = object.allowAlias ?? false; message.deprecated = object.deprecated ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseEnumValueOptions(): EnumValueOptions { - return { deprecated: false, features: undefined, debugRedact: false, uninterpretedOption: [] }; + return { deprecated: false, uninterpretedOption: [] }; } export const EnumValueOptions = { @@ -4789,12 +3894,6 @@ export const EnumValueOptions = { if (message.deprecated === true) { writer.uint32(8).bool(message.deprecated); } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(18).fork()).ldelim(); - } - if (message.debugRedact === true) { - writer.uint32(24).bool(message.debugRedact); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -4815,20 +3914,6 @@ export const EnumValueOptions = { message.deprecated = reader.bool(); continue; - case 2: - if (tag !== 18) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; - case 3: - if (tag !== 24) { - break; - } - - message.debugRedact = reader.bool(); - continue; case 999: if (tag !== 7994) { break; @@ -4848,8 +3933,6 @@ export const EnumValueOptions = { fromJSON(object: any): EnumValueOptions { return { deprecated: isSet(object.deprecated) ? Boolean(object.deprecated) : false, - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, - debugRedact: isSet(object.debugRedact) ? Boolean(object.debugRedact) : false, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4861,12 +3944,6 @@ export const EnumValueOptions = { if (message.deprecated === true) { obj.deprecated = message.deprecated; } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } - if (message.debugRedact === true) { - obj.debugRedact = message.debugRedact; - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4879,24 +3956,17 @@ export const EnumValueOptions = { fromPartial, I>>(object: I): EnumValueOptions { const message = createBaseEnumValueOptions(); message.deprecated = object.deprecated ?? false; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; - message.debugRedact = object.debugRedact ?? false; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseServiceOptions(): ServiceOptions { - return { features: undefined, deprecated: false, uninterpretedOption: [] }; + return { deprecated: false, uninterpretedOption: [] }; } export const ServiceOptions = { encode(message: ServiceOptions, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(274).fork()).ldelim(); - } if (message.deprecated === true) { writer.uint32(264).bool(message.deprecated); } @@ -4913,13 +3983,6 @@ export const ServiceOptions = { while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { - case 34: - if (tag !== 274) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 33: if (tag !== 264) { break; @@ -4945,7 +4008,6 @@ export const ServiceOptions = { fromJSON(object: any): ServiceOptions { return { - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, deprecated: isSet(object.deprecated) ? Boolean(object.deprecated) : false, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) @@ -4955,9 +4017,6 @@ export const ServiceOptions = { toJSON(message: ServiceOptions): unknown { const obj: any = {}; - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.deprecated === true) { obj.deprecated = message.deprecated; } @@ -4972,9 +4031,6 @@ export const ServiceOptions = { }, fromPartial, I>>(object: I): ServiceOptions { const message = createBaseServiceOptions(); - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.deprecated = object.deprecated ?? false; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; @@ -4982,7 +4038,7 @@ export const ServiceOptions = { }; function createBaseMethodOptions(): MethodOptions { - return { deprecated: false, idempotencyLevel: 0, features: undefined, uninterpretedOption: [] }; + return { deprecated: false, idempotencyLevel: 0, uninterpretedOption: [] }; } export const MethodOptions = { @@ -4993,9 +4049,6 @@ export const MethodOptions = { if (message.idempotencyLevel !== 0) { writer.uint32(272).int32(message.idempotencyLevel); } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(282).fork()).ldelim(); - } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -5023,13 +4076,6 @@ export const MethodOptions = { message.idempotencyLevel = reader.int32() as any; continue; - case 35: - if (tag !== 282) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; case 999: if (tag !== 7994) { break; @@ -5052,7 +4098,6 @@ export const MethodOptions = { idempotencyLevel: isSet(object.idempotencyLevel) ? methodOptions_IdempotencyLevelFromJSON(object.idempotencyLevel) : 0, - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -5067,9 +4112,6 @@ export const MethodOptions = { if (message.idempotencyLevel !== 0) { obj.idempotencyLevel = methodOptions_IdempotencyLevelToJSON(message.idempotencyLevel); } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -5083,9 +4125,6 @@ export const MethodOptions = { const message = createBaseMethodOptions(); message.deprecated = object.deprecated ?? false; message.idempotencyLevel = object.idempotencyLevel ?? 0; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -5322,320 +4361,6 @@ export const UninterpretedOption_NamePart = { }, }; -function createBaseFeatureSet(): FeatureSet { - return { - fieldPresence: 0, - enumType: 0, - repeatedFieldEncoding: 0, - utf8Validation: 0, - messageEncoding: 0, - jsonFormat: 0, - }; -} - -export const FeatureSet = { - encode(message: FeatureSet, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.fieldPresence !== 0) { - writer.uint32(8).int32(message.fieldPresence); - } - if (message.enumType !== 0) { - writer.uint32(16).int32(message.enumType); - } - if (message.repeatedFieldEncoding !== 0) { - writer.uint32(24).int32(message.repeatedFieldEncoding); - } - if (message.utf8Validation !== 0) { - writer.uint32(32).int32(message.utf8Validation); - } - if (message.messageEncoding !== 0) { - writer.uint32(40).int32(message.messageEncoding); - } - if (message.jsonFormat !== 0) { - writer.uint32(48).int32(message.jsonFormat); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSet { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseFeatureSet(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 8) { - break; - } - - message.fieldPresence = reader.int32() as any; - continue; - case 2: - if (tag !== 16) { - break; - } - - message.enumType = reader.int32() as any; - continue; - case 3: - if (tag !== 24) { - break; - } - - message.repeatedFieldEncoding = reader.int32() as any; - continue; - case 4: - if (tag !== 32) { - break; - } - - message.utf8Validation = reader.int32() as any; - continue; - case 5: - if (tag !== 40) { - break; - } - - message.messageEncoding = reader.int32() as any; - continue; - case 6: - if (tag !== 48) { - break; - } - - message.jsonFormat = reader.int32() as any; - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): FeatureSet { - return { - fieldPresence: isSet(object.fieldPresence) ? featureSet_FieldPresenceFromJSON(object.fieldPresence) : 0, - enumType: isSet(object.enumType) ? featureSet_EnumTypeFromJSON(object.enumType) : 0, - repeatedFieldEncoding: isSet(object.repeatedFieldEncoding) - ? featureSet_RepeatedFieldEncodingFromJSON(object.repeatedFieldEncoding) - : 0, - utf8Validation: isSet(object.utf8Validation) ? featureSet_Utf8ValidationFromJSON(object.utf8Validation) : 0, - messageEncoding: isSet(object.messageEncoding) ? featureSet_MessageEncodingFromJSON(object.messageEncoding) : 0, - jsonFormat: isSet(object.jsonFormat) ? featureSet_JsonFormatFromJSON(object.jsonFormat) : 0, - }; - }, - - toJSON(message: FeatureSet): unknown { - const obj: any = {}; - if (message.fieldPresence !== 0) { - obj.fieldPresence = featureSet_FieldPresenceToJSON(message.fieldPresence); - } - if (message.enumType !== 0) { - obj.enumType = featureSet_EnumTypeToJSON(message.enumType); - } - if (message.repeatedFieldEncoding !== 0) { - obj.repeatedFieldEncoding = featureSet_RepeatedFieldEncodingToJSON(message.repeatedFieldEncoding); - } - if (message.utf8Validation !== 0) { - obj.utf8Validation = featureSet_Utf8ValidationToJSON(message.utf8Validation); - } - if (message.messageEncoding !== 0) { - obj.messageEncoding = featureSet_MessageEncodingToJSON(message.messageEncoding); - } - if (message.jsonFormat !== 0) { - obj.jsonFormat = featureSet_JsonFormatToJSON(message.jsonFormat); - } - return obj; - }, - - create, I>>(base?: I): FeatureSet { - return FeatureSet.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): FeatureSet { - const message = createBaseFeatureSet(); - message.fieldPresence = object.fieldPresence ?? 0; - message.enumType = object.enumType ?? 0; - message.repeatedFieldEncoding = object.repeatedFieldEncoding ?? 0; - message.utf8Validation = object.utf8Validation ?? 0; - message.messageEncoding = object.messageEncoding ?? 0; - message.jsonFormat = object.jsonFormat ?? 0; - return message; - }, -}; - -function createBaseFeatureSetDefaults(): FeatureSetDefaults { - return { defaults: [], minimumEdition: 0, maximumEdition: 0 }; -} - -export const FeatureSetDefaults = { - encode(message: FeatureSetDefaults, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - for (const v of message.defaults) { - FeatureSetDefaults_FeatureSetEditionDefault.encode(v!, writer.uint32(10).fork()).ldelim(); - } - if (message.minimumEdition !== 0) { - writer.uint32(32).int32(message.minimumEdition); - } - if (message.maximumEdition !== 0) { - writer.uint32(40).int32(message.maximumEdition); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSetDefaults { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseFeatureSetDefaults(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 10) { - break; - } - - message.defaults.push(FeatureSetDefaults_FeatureSetEditionDefault.decode(reader, reader.uint32())); - continue; - case 4: - if (tag !== 32) { - break; - } - - message.minimumEdition = reader.int32() as any; - continue; - case 5: - if (tag !== 40) { - break; - } - - message.maximumEdition = reader.int32() as any; - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): FeatureSetDefaults { - return { - defaults: Array.isArray(object?.defaults) - ? object.defaults.map((e: any) => FeatureSetDefaults_FeatureSetEditionDefault.fromJSON(e)) - : [], - minimumEdition: isSet(object.minimumEdition) ? editionFromJSON(object.minimumEdition) : 0, - maximumEdition: isSet(object.maximumEdition) ? editionFromJSON(object.maximumEdition) : 0, - }; - }, - - toJSON(message: FeatureSetDefaults): unknown { - const obj: any = {}; - if (message.defaults?.length) { - obj.defaults = message.defaults.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.toJSON(e)); - } - if (message.minimumEdition !== 0) { - obj.minimumEdition = editionToJSON(message.minimumEdition); - } - if (message.maximumEdition !== 0) { - obj.maximumEdition = editionToJSON(message.maximumEdition); - } - return obj; - }, - - create, I>>(base?: I): FeatureSetDefaults { - return FeatureSetDefaults.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): FeatureSetDefaults { - const message = createBaseFeatureSetDefaults(); - message.defaults = object.defaults?.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(e)) || []; - message.minimumEdition = object.minimumEdition ?? 0; - message.maximumEdition = object.maximumEdition ?? 0; - return message; - }, -}; - -function createBaseFeatureSetDefaults_FeatureSetEditionDefault(): FeatureSetDefaults_FeatureSetEditionDefault { - return { edition: 0, features: undefined }; -} - -export const FeatureSetDefaults_FeatureSetEditionDefault = { - encode(message: FeatureSetDefaults_FeatureSetEditionDefault, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.edition !== 0) { - writer.uint32(24).int32(message.edition); - } - if (message.features !== undefined) { - FeatureSet.encode(message.features, writer.uint32(18).fork()).ldelim(); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSetDefaults_FeatureSetEditionDefault { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseFeatureSetDefaults_FeatureSetEditionDefault(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 3: - if (tag !== 24) { - break; - } - - message.edition = reader.int32() as any; - continue; - case 2: - if (tag !== 18) { - break; - } - - message.features = FeatureSet.decode(reader, reader.uint32()); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): FeatureSetDefaults_FeatureSetEditionDefault { - return { - edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, - features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, - }; - }, - - toJSON(message: FeatureSetDefaults_FeatureSetEditionDefault): unknown { - const obj: any = {}; - if (message.edition !== 0) { - obj.edition = editionToJSON(message.edition); - } - if (message.features !== undefined) { - obj.features = FeatureSet.toJSON(message.features); - } - return obj; - }, - - create, I>>( - base?: I, - ): FeatureSetDefaults_FeatureSetEditionDefault { - return FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>( - object: I, - ): FeatureSetDefaults_FeatureSetEditionDefault { - const message = createBaseFeatureSetDefaults_FeatureSetEditionDefault(); - message.edition = object.edition ?? 0; - message.features = (object.features !== undefined && object.features !== null) - ? FeatureSet.fromPartial(object.features) - : undefined; - return message; - }, -}; - function createBaseSourceCodeInfo(): SourceCodeInfo { return { location: [] }; } diff --git a/ui/grpc_generated/google/protobuf/timestamp.ts b/ui/grpc_generated/google/protobuf/timestamp.ts index 959778faa..560af8a4e 100644 --- a/ui/grpc_generated/google/protobuf/timestamp.ts +++ b/ui/grpc_generated/google/protobuf/timestamp.ts @@ -92,7 +92,7 @@ export const protobufPackage = "google.protobuf"; * [`strftime`](https://docs.python.org/2/library/time.html#time.strftime) with * the time format spec '%Y-%m-%dT%H:%M:%S.%fZ'. Likewise, in Java, one can use * the Joda Time's [`ISODateTimeFormat.dateTime()`]( - * http://joda-time.sourceforge.net/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime() + * http://www.joda.org/joda-time/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime%2D%2D * ) to obtain a formatter capable of generating timestamps in this format. */ export interface Timestamp { diff --git a/ui/grpc_generated/peers.ts b/ui/grpc_generated/peers.ts index 8d4b9b4ee..44fbc0338 100644 --- a/ui/grpc_generated/peers.ts +++ b/ui/grpc_generated/peers.ts @@ -181,6 +181,10 @@ export interface Peer { eventhubGroupConfig?: EventHubGroupConfig | undefined; } +export interface PostgresPeerConfigs { + configs: PostgresConfig[]; +} + function createBaseSnowflakeConfig(): SnowflakeConfig { return { accountId: "", @@ -1736,6 +1740,65 @@ export const Peer = { }, }; +function createBasePostgresPeerConfigs(): PostgresPeerConfigs { + return { configs: [] }; +} + +export const PostgresPeerConfigs = { + encode(message: PostgresPeerConfigs, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + for (const v of message.configs) { + PostgresConfig.encode(v!, writer.uint32(10).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): PostgresPeerConfigs { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePostgresPeerConfigs(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.configs.push(PostgresConfig.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): PostgresPeerConfigs { + return { + configs: Array.isArray(object?.configs) ? object.configs.map((e: any) => PostgresConfig.fromJSON(e)) : [], + }; + }, + + toJSON(message: PostgresPeerConfigs): unknown { + const obj: any = {}; + if (message.configs?.length) { + obj.configs = message.configs.map((e) => PostgresConfig.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): PostgresPeerConfigs { + return PostgresPeerConfigs.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PostgresPeerConfigs { + const message = createBasePostgresPeerConfigs(); + message.configs = object.configs?.map((e) => PostgresConfig.fromPartial(e)) || []; + return message; + }, +}; + declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined; From 5b3ff68ef5f72ba98b6b4af13fb0f314e62a5f80 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 19:45:57 +0530 Subject: [PATCH 2/8] moves getpgconfigs to activity, close ticker and pools --- flow/activities/flowable.go | 69 ++++++++++++++--- flow/cmd/api.go | 14 +--- flow/cmd/peer_data.go | 25 ------ flow/generated/protos/peers.pb.go | 118 +++++++---------------------- flow/workflows/heartbeat_flow.go | 6 +- nexus/pt/src/peerdb_peers.rs | 6 -- nexus/pt/src/peerdb_peers.serde.rs | 95 ----------------------- protos/peers.proto | 4 - ui/app/mirrors/create/schema.ts | 12 +-- ui/grpc_generated/peers.ts | 63 --------------- 10 files changed, 95 insertions(+), 317 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 46e1d3cda..1143e42d3 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -2,6 +2,7 @@ package activities import ( "context" + "database/sql" "errors" "fmt" "sync" @@ -12,15 +13,17 @@ import ( connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" + catalog "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" log "github.com/sirupsen/logrus" "go.temporal.io/sdk/activity" "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" ) // CheckConnectionResult is the result of a CheckConnection call. @@ -660,9 +663,46 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown return nil } -func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context, configs []*protos.PostgresConfig) error { - log.Info("sending walheartbeat every 10 minutes") - ticker := time.NewTicker(10 * time.Second) +func GetPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, error) { + var peerOptions sql.RawBytes + catalogPool, catalogErr := catalog.GetCatalogConnectionPoolFromEnv() + if catalogErr != nil { + return nil, fmt.Errorf("error getting catalog connection pool: %w", catalogErr) + } + defer catalogPool.Close() + + optionRows, err := catalogPool.Query(ctx, "SELECT options FROM peers WHERE type=3") + if err != nil { + return nil, err + } + defer optionRows.Close() + var peerConfigs []*protos.PostgresConfig + for optionRows.Next() { + err := optionRows.Scan(&peerOptions) + if err != nil { + return nil, err + } + var pgPeerConfig protos.PostgresConfig + unmarshalErr := proto.Unmarshal(peerOptions, &pgPeerConfig) + if unmarshalErr != nil { + return nil, unmarshalErr + } + peerConfigs = append(peerConfigs, &pgPeerConfig) + } + return peerConfigs, nil +} + +func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { + sendTimeout := 10 * time.Second + ticker := time.NewTicker(sendTimeout) + defer ticker.Stop() + + pgConfigs, err := GetPostgresPeerConfigs(ctx) + if err != nil { + return fmt.Errorf("error getting postgres peer configs: %w", err) + } + + activity.RecordHeartbeat(ctx, "sending walheartbeat every 10 minutes") for { select { case <-ctx.Done(): @@ -677,19 +717,24 @@ func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context, configs []*prot END; ` // run above command for each Postgres peer - for _, pgConfig := range configs { - peerPool, poolErr := pgxpool.New(ctx, utils.GetPGConnectionString(pgConfig)) - if poolErr != nil { - return fmt.Errorf("error creating pool for postgres peer with host %v: %w", pgConfig.Host, poolErr) + for _, pgConfig := range pgConfigs { + peerConn, peerErr := pgx.Connect(ctx, utils.GetPGConnectionString(pgConfig)) + if peerErr != nil { + return fmt.Errorf("error creating pool for postgres peer with host %v: %w", pgConfig.Host, peerErr) } - _, err := peerPool.Exec(ctx, command) - if err == nil { - log.Infof("sent wal heartbeat to postgres peer with host %v and port %v", pgConfig.Host, pgConfig.Port) - } else { + _, err := peerConn.Exec(ctx, command) + if err != nil { log.Warnf("warning: could not send walheartbeat to host %v: %v", pgConfig.Host, err) } + + closeErr := peerConn.Close(ctx) + if closeErr != nil { + return fmt.Errorf("error closing postgres connection for host %v: %w", pgConfig.Host, closeErr) + } } + ticker.Stop() + ticker = time.NewTicker(sendTimeout) } } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 3e2796c81..cfd79ed5c 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -71,12 +71,11 @@ func KillExistingHeartbeatFlows(ctx context.Context, tc client.Client, namespace if err != nil { return fmt.Errorf("unable to list workflows: %w", err) } - log.Info("Terminating pre-existing heartbeat flows") + log.Info("Requesting cancellation of pre-existing heartbeat flows") for _, workflow := range listRes.Executions { - log.Info("Terminating workflow: ", workflow.Execution.WorkflowId) - err := tc.TerminateWorkflow(ctx, - workflow.Execution.WorkflowId, workflow.Execution.RunId, - "Terminate pre-existing heartbeat flows before kicking off the current one") + log.Info("Cancelling workflow: ", workflow.Execution.WorkflowId) + err := tc.CancelWorkflow(ctx, + workflow.Execution.WorkflowId, workflow.Execution.RunId) if err != nil && err.Error() != "workflow execution already completed" { return fmt.Errorf("unable to terminate workflow: %w", err) } @@ -127,16 +126,11 @@ func APIMain(args *APIServerParams) error { ID: workflowID, TaskQueue: shared.PeerFlowTaskQueue, } - heartbeatRequest, err := flowHandler.GetPostgresPeerConfigs(ctx) - if err != nil { - return fmt.Errorf("unable to get postgres peer configs: %w", err) - } _, err = flowHandler.temporalClient.ExecuteWorkflow( ctx, // context workflowOptions, // workflow start options peerflow.HeartbeatFlowWorkflow, // workflow function - heartbeatRequest, // workflow input ) if err != nil { return fmt.Errorf("unable to start heartbeat workflow: %w", err) diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 7e70eed33..14c8ba43c 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -299,28 +299,3 @@ func (h *FlowRequestHandler) GetStatInfo( StatData: statInfoRows, }, nil } - -func (h *FlowRequestHandler) GetPostgresPeerConfigs(ctx context.Context) (*protos.PostgresPeerConfigs, error) { - var peerOptions sql.RawBytes - optionRows, err := h.pool.Query(ctx, "SELECT options FROM peers WHERE type=3") - if err != nil { - return nil, err - } - defer optionRows.Close() - var peerConfigs []*protos.PostgresConfig - for optionRows.Next() { - err := optionRows.Scan(&peerOptions) - if err != nil { - return nil, err - } - var pgPeerConfig protos.PostgresConfig - unmarshalErr := proto.Unmarshal(peerOptions, &pgPeerConfig) - if unmarshalErr != nil { - return nil, unmarshalErr - } - peerConfigs = append(peerConfigs, &pgPeerConfig) - } - return &protos.PostgresPeerConfigs{ - Configs: peerConfigs, - }, nil -} diff --git a/flow/generated/protos/peers.pb.go b/flow/generated/protos/peers.pb.go index b50d2a7ba..6ac88a4c1 100644 --- a/flow/generated/protos/peers.pb.go +++ b/flow/generated/protos/peers.pb.go @@ -1024,53 +1024,6 @@ func (*Peer_SqlserverConfig) isPeer_Config() {} func (*Peer_EventhubGroupConfig) isPeer_Config() {} -type PostgresPeerConfigs struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Configs []*PostgresConfig `protobuf:"bytes,1,rep,name=configs,proto3" json:"configs,omitempty"` -} - -func (x *PostgresPeerConfigs) Reset() { - *x = PostgresPeerConfigs{} - if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *PostgresPeerConfigs) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*PostgresPeerConfigs) ProtoMessage() {} - -func (x *PostgresPeerConfigs) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[9] - 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 PostgresPeerConfigs.ProtoReflect.Descriptor instead. -func (*PostgresPeerConfigs) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{9} -} - -func (x *PostgresPeerConfigs) GetConfigs() []*PostgresConfig { - if x != nil { - return x.Configs - } - return nil -} - var File_peers_proto protoreflect.FileDescriptor var file_peers_proto_rawDesc = []byte{ @@ -1262,27 +1215,22 @@ var file_peers_proto_rawDesc = []byte{ 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x13, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x22, 0x4d, 0x0a, 0x13, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x50, 0x65, 0x65, 0x72, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x36, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x2a, - 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, 0x47, - 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, 0x46, - 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, 0x10, - 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, 0x12, - 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x12, 0x06, 0x0a, - 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x53, 0x45, 0x52, 0x56, - 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, - 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x42, 0x0a, 0x50, 0x65, - 0x65, 0x72, 0x73, 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, 0x50, 0x65, 0x65, 0x72, 0x73, - 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0xe2, 0x02, - 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x5c, 0x47, 0x50, 0x42, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x2a, 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, + 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, + 0x46, 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, + 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, + 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x12, 0x06, + 0x0a, 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x53, 0x45, 0x52, + 0x56, 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, + 0x42, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x42, 0x0a, 0x50, + 0x65, 0x65, 0x72, 0x73, 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, 0x50, 0x65, 0x65, 0x72, + 0x73, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0xe2, + 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x5c, 0x47, 0x50, + 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1298,7 +1246,7 @@ func file_peers_proto_rawDescGZIP() []byte { } var file_peers_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 11) +var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 10) var file_peers_proto_goTypes = []interface{}{ (DBType)(0), // 0: peerdb_peers.DBType (*SnowflakeConfig)(nil), // 1: peerdb_peers.SnowflakeConfig @@ -1310,12 +1258,11 @@ var file_peers_proto_goTypes = []interface{}{ (*S3Config)(nil), // 7: peerdb_peers.S3Config (*SqlServerConfig)(nil), // 8: peerdb_peers.SqlServerConfig (*Peer)(nil), // 9: peerdb_peers.Peer - (*PostgresPeerConfigs)(nil), // 10: peerdb_peers.PostgresPeerConfigs - nil, // 11: peerdb_peers.EventHubGroupConfig.EventhubsEntry + nil, // 10: peerdb_peers.EventHubGroupConfig.EventhubsEntry } var file_peers_proto_depIdxs = []int32{ 4, // 0: peerdb_peers.EventHubConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig - 11, // 1: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry + 10, // 1: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry 4, // 2: peerdb_peers.EventHubGroupConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig 4, // 3: peerdb_peers.S3Config.metadata_db:type_name -> peerdb_peers.PostgresConfig 0, // 4: peerdb_peers.Peer.type:type_name -> peerdb_peers.DBType @@ -1327,13 +1274,12 @@ var file_peers_proto_depIdxs = []int32{ 7, // 10: peerdb_peers.Peer.s3_config:type_name -> peerdb_peers.S3Config 8, // 11: peerdb_peers.Peer.sqlserver_config:type_name -> peerdb_peers.SqlServerConfig 6, // 12: peerdb_peers.Peer.eventhub_group_config:type_name -> peerdb_peers.EventHubGroupConfig - 4, // 13: peerdb_peers.PostgresPeerConfigs.configs:type_name -> peerdb_peers.PostgresConfig - 5, // 14: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig - 15, // [15:15] is the sub-list for method output_type - 15, // [15:15] is the sub-list for method input_type - 15, // [15:15] is the sub-list for extension type_name - 15, // [15:15] is the sub-list for extension extendee - 0, // [0:15] is the sub-list for field type_name + 5, // 13: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig + 14, // [14:14] is the sub-list for method output_type + 14, // [14:14] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name } func init() { file_peers_proto_init() } @@ -1450,18 +1396,6 @@ func file_peers_proto_init() { return nil } } - file_peers_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresPeerConfigs); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } } file_peers_proto_msgTypes[0].OneofWrappers = []interface{}{} file_peers_proto_msgTypes[3].OneofWrappers = []interface{}{} @@ -1482,7 +1416,7 @@ func file_peers_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_peers_proto_rawDesc, NumEnums: 1, - NumMessages: 11, + NumMessages: 10, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/workflows/heartbeat_flow.go b/flow/workflows/heartbeat_flow.go index b24a4bf35..80e89745d 100644 --- a/flow/workflows/heartbeat_flow.go +++ b/flow/workflows/heartbeat_flow.go @@ -3,19 +3,17 @@ package peerflow import ( "time" - "github.com/PeerDB-io/peer-flow/generated/protos" "go.temporal.io/sdk/workflow" ) // HeartbeatFlowWorkflow is the workflow that sets up heartbeat sending. -func HeartbeatFlowWorkflow(ctx workflow.Context, - config *protos.PostgresPeerConfigs) error { +func HeartbeatFlowWorkflow(ctx workflow.Context) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 7 * 24 * time.Hour, }) - heartbeatFuture := workflow.ExecuteActivity(ctx, flowable.SendWALHeartbeat, config.Configs) + heartbeatFuture := workflow.ExecuteActivity(ctx, flowable.SendWALHeartbeat) if err := heartbeatFuture.Get(ctx, nil); err != nil { return err } diff --git a/nexus/pt/src/peerdb_peers.rs b/nexus/pt/src/peerdb_peers.rs index 1bf1b9732..017a36b8a 100644 --- a/nexus/pt/src/peerdb_peers.rs +++ b/nexus/pt/src/peerdb_peers.rs @@ -181,12 +181,6 @@ pub mod peer { EventhubGroupConfig(super::EventHubGroupConfig), } } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct PostgresPeerConfigs { - #[prost(message, repeated, tag="1")] - pub configs: ::prost::alloc::vec::Vec, -} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum DbType { diff --git a/nexus/pt/src/peerdb_peers.serde.rs b/nexus/pt/src/peerdb_peers.serde.rs index af6f53fa6..2cdf144f4 100644 --- a/nexus/pt/src/peerdb_peers.serde.rs +++ b/nexus/pt/src/peerdb_peers.serde.rs @@ -1306,101 +1306,6 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { deserializer.deserialize_struct("peerdb_peers.PostgresConfig", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for PostgresPeerConfigs { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if !self.configs.is_empty() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("peerdb_peers.PostgresPeerConfigs", len)?; - if !self.configs.is_empty() { - struct_ser.serialize_field("configs", &self.configs)?; - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for PostgresPeerConfigs { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "configs", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - Configs, - __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 { - "configs" => Ok(GeneratedField::Configs), - _ => Ok(GeneratedField::__SkipField__), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = PostgresPeerConfigs; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_peers.PostgresPeerConfigs") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut configs__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::Configs => { - if configs__.is_some() { - return Err(serde::de::Error::duplicate_field("configs")); - } - configs__ = Some(map.next_value()?); - } - GeneratedField::__SkipField__ => { - let _ = map.next_value::()?; - } - } - } - Ok(PostgresPeerConfigs { - configs: configs__.unwrap_or_default(), - }) - } - } - deserializer.deserialize_struct("peerdb_peers.PostgresPeerConfigs", FIELDS, GeneratedVisitor) - } -} impl serde::Serialize for S3Config { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/peers.proto b/protos/peers.proto index 5b7dea115..fe343d906 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -113,7 +113,3 @@ message Peer { EventHubGroupConfig eventhub_group_config = 10; } } - -message PostgresPeerConfigs { - repeated PostgresConfig configs = 1; -} \ No newline at end of file diff --git a/ui/app/mirrors/create/schema.ts b/ui/app/mirrors/create/schema.ts index 3400ba459..19f2eaeb2 100644 --- a/ui/app/mirrors/create/schema.ts +++ b/ui/app/mirrors/create/schema.ts @@ -58,24 +58,24 @@ export const cdcSchema = z.object({ .optional(), snapshotNumRowsPerPartition: z .number({ - invalid_type_error: 'Snapshow rows per partition must be a number', + invalid_type_error: 'Snapshot rows per partition must be a number', }) .int() - .min(1, 'Snapshow rows per partition must be a positive integer') + .min(1, 'Snapshot rows per partition must be a positive integer') .optional(), snapshotMaxParallelWorkers: z .number({ - invalid_type_error: 'Snapshow max workers must be a number', + invalid_type_error: 'Snapshot max workers must be a number', }) .int() - .min(1, 'Snapshow max workers must be a positive integer') + .min(1, 'Snapshot max workers must be a positive integer') .optional(), snapshotNumTablesInParallel: z .number({ - invalid_type_error: 'Snapshow parallel tables must be a number', + invalid_type_error: 'Snapshot parallel tables must be a number', }) .int() - .min(1, 'Snapshow parallel tables must be a positive integer') + .min(1, 'Snapshot parallel tables must be a positive integer') .optional(), snapshotStagingPath: z .string({ diff --git a/ui/grpc_generated/peers.ts b/ui/grpc_generated/peers.ts index 44fbc0338..8d4b9b4ee 100644 --- a/ui/grpc_generated/peers.ts +++ b/ui/grpc_generated/peers.ts @@ -181,10 +181,6 @@ export interface Peer { eventhubGroupConfig?: EventHubGroupConfig | undefined; } -export interface PostgresPeerConfigs { - configs: PostgresConfig[]; -} - function createBaseSnowflakeConfig(): SnowflakeConfig { return { accountId: "", @@ -1740,65 +1736,6 @@ export const Peer = { }, }; -function createBasePostgresPeerConfigs(): PostgresPeerConfigs { - return { configs: [] }; -} - -export const PostgresPeerConfigs = { - encode(message: PostgresPeerConfigs, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - for (const v of message.configs) { - PostgresConfig.encode(v!, writer.uint32(10).fork()).ldelim(); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): PostgresPeerConfigs { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBasePostgresPeerConfigs(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 10) { - break; - } - - message.configs.push(PostgresConfig.decode(reader, reader.uint32())); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): PostgresPeerConfigs { - return { - configs: Array.isArray(object?.configs) ? object.configs.map((e: any) => PostgresConfig.fromJSON(e)) : [], - }; - }, - - toJSON(message: PostgresPeerConfigs): unknown { - const obj: any = {}; - if (message.configs?.length) { - obj.configs = message.configs.map((e) => PostgresConfig.toJSON(e)); - } - return obj; - }, - - create, I>>(base?: I): PostgresPeerConfigs { - return PostgresPeerConfigs.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): PostgresPeerConfigs { - const message = createBasePostgresPeerConfigs(); - message.configs = object.configs?.map((e) => PostgresConfig.fromPartial(e)) || []; - return message; - }, -}; - declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined; From 04e566a1006946d0f62db7a062ce784169001209 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 19:47:37 +0530 Subject: [PATCH 3/8] change timeout to 10 minutes --- flow/activities/flowable.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 1143e42d3..c452f6d7a 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -693,7 +693,7 @@ func GetPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, erro } func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { - sendTimeout := 10 * time.Second + sendTimeout := 10 * time.Minute ticker := time.NewTicker(sendTimeout) defer ticker.Stop() From e67b3333519fafeaf6fb780a52e5113d7bb5d149 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 19:49:56 +0530 Subject: [PATCH 4/8] remove search attributes --- flow/cmd/handler.go | 3 --- flow/workflows/cdc_flow.go | 14 +------------- flow/workflows/qrep_flow.go | 3 --- 3 files changed, 1 insertion(+), 19 deletions(-) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index c02140407..13ce8115b 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -309,9 +309,6 @@ func (h *FlowRequestHandler) ShutdownFlow( workflowOptions := client.StartWorkflowOptions{ ID: workflowID, TaskQueue: shared.PeerFlowTaskQueue, - SearchAttributes: map[string]interface{}{ - "MirrorName": req.FlowJobName, - }, } dropFlowHandle, err := h.temporalClient.ExecuteWorkflow( ctx, // context diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 3abdf6f5e..753cff380 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -189,10 +189,6 @@ func CDCFlowWorkflowWithConfig( } } - searchAttributes := map[string]interface{}{ - "MirrorName": cfg.FlowJobName, - } - // start the SetupFlow workflow as a child workflow, and wait for it to complete // it should return the table schema for the source peer setupFlowID, err := GetChildWorkflowID(ctx, "setup-flow", cfg.FlowJobName) @@ -205,7 +201,6 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - SearchAttributes: searchAttributes, } setupFlowCtx := workflow.WithChildOptions(ctx, childSetupFlowOpts) setupFlowFuture := workflow.ExecuteChildWorkflow(setupFlowCtx, SetupFlowWorkflow, cfg) @@ -225,8 +220,7 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - TaskQueue: shared.SnapshotFlowTaskQueue, - SearchAttributes: searchAttributes, + TaskQueue: shared.SnapshotFlowTaskQueue, } snapshotFlowCtx := workflow.WithChildOptions(ctx, childSnapshotFlowOpts) snapshotFlowFuture := workflow.ExecuteChildWorkflow(snapshotFlowCtx, SnapshotFlowWorkflow, cfg) @@ -323,10 +317,6 @@ func CDCFlowWorkflowWithConfig( return state, err } - searchAttributes := map[string]interface{}{ - "MirrorName": cfg.FlowJobName, - } - // execute the sync flow as a child workflow childSyncFlowOpts := workflow.ChildWorkflowOptions{ WorkflowID: syncFlowID, @@ -334,7 +324,6 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - SearchAttributes: searchAttributes, } ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) syncFlowOptions.RelationMessageMapping = *state.RelationMessageMapping @@ -367,7 +356,6 @@ func CDCFlowWorkflowWithConfig( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - SearchAttributes: searchAttributes, } ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index c8a7c1023..3b8e77a68 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -206,9 +206,6 @@ func (q *QRepFlowExecution) startChildWorkflow( RetryPolicy: &temporal.RetryPolicy{ MaximumAttempts: 20, }, - SearchAttributes: map[string]interface{}{ - "MirrorName": q.config.FlowJobName, - }, }) future := workflow.ExecuteChildWorkflow( From 3fbc1cefa1b28adac1855e3dc058bf3d50624173 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 19:54:04 +0530 Subject: [PATCH 5/8] revert dev docker to original --- docker-compose-dev.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index 4c8297d18..840bd556f 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -63,7 +63,6 @@ services: - POSTGRES_SEEDS=catalog - DYNAMIC_CONFIG_FILE_PATH=config/dynamicconfig/development-sql.yaml image: temporalio/auto-setup:1.22 - command: sh -c 'temporal operator search-attribute create --name MirrorName --type Text' ports: - 7233:7233 volumes: From 62ef43cf1ed502d6fcef6d3c8080670045177635 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 20:15:36 +0530 Subject: [PATCH 6/8] suggested changes --- flow/activities/flowable.go | 6 +++--- flow/cmd/api.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index c452f6d7a..d811f2cd3 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -663,7 +663,7 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown return nil } -func GetPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, error) { +func getPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, error) { var peerOptions sql.RawBytes catalogPool, catalogErr := catalog.GetCatalogConnectionPoolFromEnv() if catalogErr != nil { @@ -671,7 +671,7 @@ func GetPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, erro } defer catalogPool.Close() - optionRows, err := catalogPool.Query(ctx, "SELECT options FROM peers WHERE type=3") + optionRows, err := catalogPool.Query(ctx, "SELECT options FROM peers WHERE type=$1", protos.DBType_POSTGRES) if err != nil { return nil, err } @@ -697,7 +697,7 @@ func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { ticker := time.NewTicker(sendTimeout) defer ticker.Stop() - pgConfigs, err := GetPostgresPeerConfigs(ctx) + pgConfigs, err := getPostgresPeerConfigs(ctx) if err != nil { return fmt.Errorf("error getting postgres peer configs: %w", err) } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index cfd79ed5c..67273be8a 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -62,7 +62,7 @@ func setupGRPCGatewayServer(args *APIServerParams) (*http.Server, error) { return server, nil } -func KillExistingHeartbeatFlows(ctx context.Context, tc client.Client, namespace string) error { +func killExistingHeartbeatFlows(ctx context.Context, tc client.Client, namespace string) error { listRes, err := tc.ListWorkflow(ctx, &workflowservice.ListWorkflowExecutionsRequest{ Namespace: namespace, @@ -116,7 +116,7 @@ func APIMain(args *APIServerParams) error { flowHandler := NewFlowRequestHandler(tc, catalogConn) defer flowHandler.Close() - err = KillExistingHeartbeatFlows(ctx, tc, args.TemporalNamespace) + err = killExistingHeartbeatFlows(ctx, tc, args.TemporalNamespace) if err != nil { return fmt.Errorf("unable to kill existing heartbeat flows: %w", err) } From b1c425601a7a43d3141cd7b78eb7b62bc5e5f63b Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 20:59:31 +0530 Subject: [PATCH 7/8] undo generated file changes, add better logging --- flow/activities/flowable.go | 34 +- .../google/protobuf/descriptor.ts | 1379 ++++++++++++++++- .../google/protobuf/timestamp.ts | 2 +- 3 files changed, 1350 insertions(+), 65 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index d811f2cd3..d7900a9b5 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -663,7 +663,7 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown return nil } -func getPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, error) { +func getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { var peerOptions sql.RawBytes catalogPool, catalogErr := catalog.GetCatalogConnectionPoolFromEnv() if catalogErr != nil { @@ -671,14 +671,16 @@ func getPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, erro } defer catalogPool.Close() - optionRows, err := catalogPool.Query(ctx, "SELECT options FROM peers WHERE type=$1", protos.DBType_POSTGRES) + optionRows, err := catalogPool.Query(ctx, + "SELECT name, options FROM peers WHERE type=$1", protos.DBType_POSTGRES) if err != nil { return nil, err } defer optionRows.Close() - var peerConfigs []*protos.PostgresConfig + var peerName string + var postgresPeers []*protos.Peer for optionRows.Next() { - err := optionRows.Scan(&peerOptions) + err := optionRows.Scan(&peerName, &peerOptions) if err != nil { return nil, err } @@ -687,17 +689,21 @@ func getPostgresPeerConfigs(ctx context.Context) ([]*protos.PostgresConfig, erro if unmarshalErr != nil { return nil, unmarshalErr } - peerConfigs = append(peerConfigs, &pgPeerConfig) + postgresPeers = append(postgresPeers, &protos.Peer{ + Name: peerName, + Type: protos.DBType_POSTGRES, + Config: &protos.Peer_PostgresConfig{PostgresConfig: &pgPeerConfig}, + }) } - return peerConfigs, nil + return postgresPeers, nil } func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { - sendTimeout := 10 * time.Minute + sendTimeout := 10 * time.Second ticker := time.NewTicker(sendTimeout) defer ticker.Stop() - pgConfigs, err := getPostgresPeerConfigs(ctx) + pgPeers, err := getPostgresPeerConfigs(ctx) if err != nil { return fmt.Errorf("error getting postgres peer configs: %w", err) } @@ -717,21 +723,25 @@ func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { END; ` // run above command for each Postgres peer - for _, pgConfig := range pgConfigs { + for _, pgPeer := range pgPeers { + pgConfig := pgPeer.GetPostgresConfig() peerConn, peerErr := pgx.Connect(ctx, utils.GetPGConnectionString(pgConfig)) if peerErr != nil { - return fmt.Errorf("error creating pool for postgres peer with host %v: %w", pgConfig.Host, peerErr) + return fmt.Errorf("error creating pool for postgres peer %v with host %v: %w", + pgPeer.Name, pgConfig.Host, peerErr) } _, err := peerConn.Exec(ctx, command) if err != nil { - log.Warnf("warning: could not send walheartbeat to host %v: %v", pgConfig.Host, err) + log.Warnf("warning: could not send walheartbeat to peer %v: %v", pgPeer.Name, err) } closeErr := peerConn.Close(ctx) if closeErr != nil { - return fmt.Errorf("error closing postgres connection for host %v: %w", pgConfig.Host, closeErr) + return fmt.Errorf("error closing postgres connection for peer %v with host %v: %w", + pgPeer.Name, pgConfig.Host, closeErr) } + log.Infof("sent walheartbeat to peer %v", pgPeer.Name) } ticker.Stop() ticker = time.NewTicker(sendTimeout) diff --git a/ui/grpc_generated/google/protobuf/descriptor.ts b/ui/grpc_generated/google/protobuf/descriptor.ts index 0ebf9063a..680c10f4d 100644 --- a/ui/grpc_generated/google/protobuf/descriptor.ts +++ b/ui/grpc_generated/google/protobuf/descriptor.ts @@ -4,6 +4,98 @@ import _m0 from "protobufjs/minimal"; export const protobufPackage = "google.protobuf"; +/** The full set of known editions. */ +export enum Edition { + /** EDITION_UNKNOWN - A placeholder for an unknown edition value. */ + EDITION_UNKNOWN = 0, + /** + * EDITION_PROTO2 - Legacy syntax "editions". These pre-date editions, but behave much like + * distinct editions. These can't be used to specify the edition of proto + * files, but feature definitions must supply proto2/proto3 defaults for + * backwards compatibility. + */ + EDITION_PROTO2 = 998, + EDITION_PROTO3 = 999, + /** + * EDITION_2023 - Editions that have been released. The specific values are arbitrary and + * should not be depended on, but they will always be time-ordered for easy + * comparison. + */ + EDITION_2023 = 1000, + /** + * EDITION_1_TEST_ONLY - Placeholder editions for testing feature resolution. These should not be + * used or relyed on outside of tests. + */ + EDITION_1_TEST_ONLY = 1, + EDITION_2_TEST_ONLY = 2, + EDITION_99997_TEST_ONLY = 99997, + EDITION_99998_TEST_ONLY = 99998, + EDITION_99999_TEST_ONLY = 99999, + UNRECOGNIZED = -1, +} + +export function editionFromJSON(object: any): Edition { + switch (object) { + case 0: + case "EDITION_UNKNOWN": + return Edition.EDITION_UNKNOWN; + case 998: + case "EDITION_PROTO2": + return Edition.EDITION_PROTO2; + case 999: + case "EDITION_PROTO3": + return Edition.EDITION_PROTO3; + case 1000: + case "EDITION_2023": + return Edition.EDITION_2023; + case 1: + case "EDITION_1_TEST_ONLY": + return Edition.EDITION_1_TEST_ONLY; + case 2: + case "EDITION_2_TEST_ONLY": + return Edition.EDITION_2_TEST_ONLY; + case 99997: + case "EDITION_99997_TEST_ONLY": + return Edition.EDITION_99997_TEST_ONLY; + case 99998: + case "EDITION_99998_TEST_ONLY": + return Edition.EDITION_99998_TEST_ONLY; + case 99999: + case "EDITION_99999_TEST_ONLY": + return Edition.EDITION_99999_TEST_ONLY; + case -1: + case "UNRECOGNIZED": + default: + return Edition.UNRECOGNIZED; + } +} + +export function editionToJSON(object: Edition): string { + switch (object) { + case Edition.EDITION_UNKNOWN: + return "EDITION_UNKNOWN"; + case Edition.EDITION_PROTO2: + return "EDITION_PROTO2"; + case Edition.EDITION_PROTO3: + return "EDITION_PROTO3"; + case Edition.EDITION_2023: + return "EDITION_2023"; + case Edition.EDITION_1_TEST_ONLY: + return "EDITION_1_TEST_ONLY"; + case Edition.EDITION_2_TEST_ONLY: + return "EDITION_2_TEST_ONLY"; + case Edition.EDITION_99997_TEST_ONLY: + return "EDITION_99997_TEST_ONLY"; + case Edition.EDITION_99998_TEST_ONLY: + return "EDITION_99998_TEST_ONLY"; + case Edition.EDITION_99999_TEST_ONLY: + return "EDITION_99999_TEST_ONLY"; + case Edition.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + /** * The protocol compiler can output a FileDescriptorSet containing the .proto * files it parses. @@ -51,8 +143,8 @@ export interface FileDescriptorProto { * If `edition` is present, this value must be "editions". */ syntax: string; - /** The edition of the proto file, which is an opaque string. */ - edition: string; + /** The edition of the proto file. */ + edition: Edition; } /** Describes a message type. */ @@ -96,6 +188,84 @@ export interface DescriptorProto_ReservedRange { export interface ExtensionRangeOptions { /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; + /** + * For external users: DO NOT USE. We are in the process of open sourcing + * extension declaration and executing internal cleanups before it can be + * used externally. + */ + declaration: ExtensionRangeOptions_Declaration[]; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; + /** + * The verification state of the range. + * TODO: flip the default to DECLARATION once all empty ranges + * are marked as UNVERIFIED. + */ + verification: ExtensionRangeOptions_VerificationState; +} + +/** The verification state of the extension range. */ +export enum ExtensionRangeOptions_VerificationState { + /** DECLARATION - All the extensions of the range must be declared. */ + DECLARATION = 0, + UNVERIFIED = 1, + UNRECOGNIZED = -1, +} + +export function extensionRangeOptions_VerificationStateFromJSON(object: any): ExtensionRangeOptions_VerificationState { + switch (object) { + case 0: + case "DECLARATION": + return ExtensionRangeOptions_VerificationState.DECLARATION; + case 1: + case "UNVERIFIED": + return ExtensionRangeOptions_VerificationState.UNVERIFIED; + case -1: + case "UNRECOGNIZED": + default: + return ExtensionRangeOptions_VerificationState.UNRECOGNIZED; + } +} + +export function extensionRangeOptions_VerificationStateToJSON(object: ExtensionRangeOptions_VerificationState): string { + switch (object) { + case ExtensionRangeOptions_VerificationState.DECLARATION: + return "DECLARATION"; + case ExtensionRangeOptions_VerificationState.UNVERIFIED: + return "UNVERIFIED"; + case ExtensionRangeOptions_VerificationState.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export interface ExtensionRangeOptions_Declaration { + /** The extension number declared within the extension range. */ + number: number; + /** + * The fully-qualified name of the extension field. There must be a leading + * dot in front of the full name. + */ + fullName: string; + /** + * The fully-qualified type name of the extension field. Unlike + * Metadata.type, Declaration.type must have a leading dot for messages + * and enums. + */ + type: string; + /** + * If true, indicates that the number is reserved in the extension range, + * and any extension field with the number will fail to compile. Set this + * when a declared extension field is deleted. + */ + reserved: boolean; + /** + * If true, indicates that the extension must be defined as repeated. + * Otherwise the extension must be defined as optional. + */ + repeated: boolean; } /** Describes a field within a message. */ @@ -193,9 +363,10 @@ export enum FieldDescriptorProto_Type { TYPE_STRING = 9, /** * TYPE_GROUP - Tag-delimited aggregate. - * Group type is deprecated and not supported in proto3. However, Proto3 + * Group type is deprecated and not supported after google.protobuf. However, Proto3 * implementations should still be able to parse the group wire format and - * treat group fields as unknown fields. + * treat group fields as unknown fields. In Editions, the group wire format + * can be enabled via the `message_encoding` feature. */ TYPE_GROUP = 10, /** TYPE_MESSAGE - Length-delimited aggregate. */ @@ -323,8 +494,13 @@ export function fieldDescriptorProto_TypeToJSON(object: FieldDescriptorProto_Typ export enum FieldDescriptorProto_Label { /** LABEL_OPTIONAL - 0 is reserved for errors */ LABEL_OPTIONAL = 1, - LABEL_REQUIRED = 2, LABEL_REPEATED = 3, + /** + * LABEL_REQUIRED - The required label is only allowed in google.protobuf. In proto3 and Editions + * it's explicitly prohibited. In Editions, the `field_presence` feature + * can be used to get this behavior. + */ + LABEL_REQUIRED = 2, UNRECOGNIZED = -1, } @@ -333,12 +509,12 @@ export function fieldDescriptorProto_LabelFromJSON(object: any): FieldDescriptor case 1: case "LABEL_OPTIONAL": return FieldDescriptorProto_Label.LABEL_OPTIONAL; - case 2: - case "LABEL_REQUIRED": - return FieldDescriptorProto_Label.LABEL_REQUIRED; case 3: case "LABEL_REPEATED": return FieldDescriptorProto_Label.LABEL_REPEATED; + case 2: + case "LABEL_REQUIRED": + return FieldDescriptorProto_Label.LABEL_REQUIRED; case -1: case "UNRECOGNIZED": default: @@ -350,10 +526,10 @@ export function fieldDescriptorProto_LabelToJSON(object: FieldDescriptorProto_La switch (object) { case FieldDescriptorProto_Label.LABEL_OPTIONAL: return "LABEL_OPTIONAL"; - case FieldDescriptorProto_Label.LABEL_REQUIRED: - return "LABEL_REQUIRED"; case FieldDescriptorProto_Label.LABEL_REPEATED: return "LABEL_REPEATED"; + case FieldDescriptorProto_Label.LABEL_REQUIRED: + return "LABEL_REQUIRED"; case FieldDescriptorProto_Label.UNRECOGNIZED: default: return "UNRECOGNIZED"; @@ -547,6 +723,10 @@ export interface FileOptions { * determining the ruby package. */ rubyPackage: string; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** * The parser stores options it doesn't recognize here. * See the documentation for the "Options" section above. @@ -665,12 +845,16 @@ export interface MessageOptions { * This should only be used as a temporary measure against broken builds due * to the change in behavior for JSON field name conflicts. * - * TODO(b/261750190) This is legacy behavior we plan to remove once downstream + * TODO This is legacy behavior we plan to remove once downstream * teams have had time to migrate. * * @deprecated */ deprecatedLegacyJsonFieldConflicts: boolean; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -679,8 +863,10 @@ export interface FieldOptions { /** * The ctype option instructs the C++ code generator to use a different * representation of the field than it normally would. See the specific - * options below. This option is not yet implemented in the open source - * release -- sorry, we'll try to include it in a future version! + * options below. This option is only implemented to support use of + * [ctype=CORD] and [ctype=STRING] (the default) on non-repeated fields of + * type "bytes" in the open source release -- sorry, we'll try to include + * other types in a future version! */ ctype: FieldOptions_CType; /** @@ -688,7 +874,9 @@ export interface FieldOptions { * a more efficient representation on the wire. Rather than repeatedly * writing the tag and type for each element, the entire array is encoded as * a single length-delimited blob. In proto3, only explicit setting it to - * false will avoid using packed encoding. + * false will avoid using packed encoding. This option is prohibited in + * Editions, but the `repeated_field_encoding` feature can be used to control + * the behavior. */ packed: boolean; /** @@ -759,7 +947,12 @@ export interface FieldOptions { */ debugRedact: boolean; retention: FieldOptions_OptionRetention; - target: FieldOptions_OptionTargetType; + targets: FieldOptions_OptionTargetType[]; + editionDefaults: FieldOptions_EditionDefault[]; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -767,6 +960,14 @@ export interface FieldOptions { export enum FieldOptions_CType { /** STRING - Default mode. */ STRING = 0, + /** + * CORD - The option [ctype=CORD] may be applied to a non-repeated field of type + * "bytes". It indicates that in C++, the data should be stored in a Cord + * instead of a string. For very large strings, this may reduce memory + * fragmentation. It may also allow better performance when parsing from a + * Cord, or when parsing with aliasing enabled, as the parsed Cord may then + * alias the original buffer. + */ CORD = 1, STRING_PIECE = 2, UNRECOGNIZED = -1, @@ -977,7 +1178,17 @@ export function fieldOptions_OptionTargetTypeToJSON(object: FieldOptions_OptionT } } +export interface FieldOptions_EditionDefault { + edition: Edition; + /** Textproto value. */ + value: string; +} + export interface OneofOptions { + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1000,12 +1211,16 @@ export interface EnumOptions { * and strips underscored from the fields before comparison in proto3 only. * The new behavior takes `json_name` into account and applies to proto2 as * well. - * TODO(b/261750190) Remove this legacy behavior once downstream teams have + * TODO Remove this legacy behavior once downstream teams have * had time to migrate. * * @deprecated */ deprecatedLegacyJsonFieldConflicts: boolean; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1018,11 +1233,25 @@ export interface EnumValueOptions { * this is a formalization for deprecating enum values. */ deprecated: boolean; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; + /** + * Indicate that fields annotated with this enum value should not be printed + * out when using debug formats, e.g. when the field contains sensitive + * credentials. + */ + debugRedact: boolean; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } export interface ServiceOptions { + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** * Is this service deprecated? * Depending on the target platform, this can emit Deprecated annotations @@ -1043,6 +1272,10 @@ export interface MethodOptions { */ deprecated: boolean; idempotencyLevel: MethodOptions_IdempotencyLevel; + /** Any features defined in the specific edition. */ + features: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1127,6 +1360,294 @@ export interface UninterpretedOption_NamePart { isExtension: boolean; } +/** + * TODO Enums in C++ gencode (and potentially other languages) are + * not well scoped. This means that each of the feature enums below can clash + * with each other. The short names we've chosen maximize call-site + * readability, but leave us very open to this scenario. A future feature will + * be designed and implemented to handle this, hopefully before we ever hit a + * conflict here. + */ +export interface FeatureSet { + fieldPresence: FeatureSet_FieldPresence; + enumType: FeatureSet_EnumType; + repeatedFieldEncoding: FeatureSet_RepeatedFieldEncoding; + utf8Validation: FeatureSet_Utf8Validation; + messageEncoding: FeatureSet_MessageEncoding; + jsonFormat: FeatureSet_JsonFormat; +} + +export enum FeatureSet_FieldPresence { + FIELD_PRESENCE_UNKNOWN = 0, + EXPLICIT = 1, + IMPLICIT = 2, + LEGACY_REQUIRED = 3, + UNRECOGNIZED = -1, +} + +export function featureSet_FieldPresenceFromJSON(object: any): FeatureSet_FieldPresence { + switch (object) { + case 0: + case "FIELD_PRESENCE_UNKNOWN": + return FeatureSet_FieldPresence.FIELD_PRESENCE_UNKNOWN; + case 1: + case "EXPLICIT": + return FeatureSet_FieldPresence.EXPLICIT; + case 2: + case "IMPLICIT": + return FeatureSet_FieldPresence.IMPLICIT; + case 3: + case "LEGACY_REQUIRED": + return FeatureSet_FieldPresence.LEGACY_REQUIRED; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_FieldPresence.UNRECOGNIZED; + } +} + +export function featureSet_FieldPresenceToJSON(object: FeatureSet_FieldPresence): string { + switch (object) { + case FeatureSet_FieldPresence.FIELD_PRESENCE_UNKNOWN: + return "FIELD_PRESENCE_UNKNOWN"; + case FeatureSet_FieldPresence.EXPLICIT: + return "EXPLICIT"; + case FeatureSet_FieldPresence.IMPLICIT: + return "IMPLICIT"; + case FeatureSet_FieldPresence.LEGACY_REQUIRED: + return "LEGACY_REQUIRED"; + case FeatureSet_FieldPresence.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export enum FeatureSet_EnumType { + ENUM_TYPE_UNKNOWN = 0, + OPEN = 1, + CLOSED = 2, + UNRECOGNIZED = -1, +} + +export function featureSet_EnumTypeFromJSON(object: any): FeatureSet_EnumType { + switch (object) { + case 0: + case "ENUM_TYPE_UNKNOWN": + return FeatureSet_EnumType.ENUM_TYPE_UNKNOWN; + case 1: + case "OPEN": + return FeatureSet_EnumType.OPEN; + case 2: + case "CLOSED": + return FeatureSet_EnumType.CLOSED; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_EnumType.UNRECOGNIZED; + } +} + +export function featureSet_EnumTypeToJSON(object: FeatureSet_EnumType): string { + switch (object) { + case FeatureSet_EnumType.ENUM_TYPE_UNKNOWN: + return "ENUM_TYPE_UNKNOWN"; + case FeatureSet_EnumType.OPEN: + return "OPEN"; + case FeatureSet_EnumType.CLOSED: + return "CLOSED"; + case FeatureSet_EnumType.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export enum FeatureSet_RepeatedFieldEncoding { + REPEATED_FIELD_ENCODING_UNKNOWN = 0, + PACKED = 1, + EXPANDED = 2, + UNRECOGNIZED = -1, +} + +export function featureSet_RepeatedFieldEncodingFromJSON(object: any): FeatureSet_RepeatedFieldEncoding { + switch (object) { + case 0: + case "REPEATED_FIELD_ENCODING_UNKNOWN": + return FeatureSet_RepeatedFieldEncoding.REPEATED_FIELD_ENCODING_UNKNOWN; + case 1: + case "PACKED": + return FeatureSet_RepeatedFieldEncoding.PACKED; + case 2: + case "EXPANDED": + return FeatureSet_RepeatedFieldEncoding.EXPANDED; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_RepeatedFieldEncoding.UNRECOGNIZED; + } +} + +export function featureSet_RepeatedFieldEncodingToJSON(object: FeatureSet_RepeatedFieldEncoding): string { + switch (object) { + case FeatureSet_RepeatedFieldEncoding.REPEATED_FIELD_ENCODING_UNKNOWN: + return "REPEATED_FIELD_ENCODING_UNKNOWN"; + case FeatureSet_RepeatedFieldEncoding.PACKED: + return "PACKED"; + case FeatureSet_RepeatedFieldEncoding.EXPANDED: + return "EXPANDED"; + case FeatureSet_RepeatedFieldEncoding.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export enum FeatureSet_Utf8Validation { + UTF8_VALIDATION_UNKNOWN = 0, + NONE = 1, + VERIFY = 2, + UNRECOGNIZED = -1, +} + +export function featureSet_Utf8ValidationFromJSON(object: any): FeatureSet_Utf8Validation { + switch (object) { + case 0: + case "UTF8_VALIDATION_UNKNOWN": + return FeatureSet_Utf8Validation.UTF8_VALIDATION_UNKNOWN; + case 1: + case "NONE": + return FeatureSet_Utf8Validation.NONE; + case 2: + case "VERIFY": + return FeatureSet_Utf8Validation.VERIFY; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_Utf8Validation.UNRECOGNIZED; + } +} + +export function featureSet_Utf8ValidationToJSON(object: FeatureSet_Utf8Validation): string { + switch (object) { + case FeatureSet_Utf8Validation.UTF8_VALIDATION_UNKNOWN: + return "UTF8_VALIDATION_UNKNOWN"; + case FeatureSet_Utf8Validation.NONE: + return "NONE"; + case FeatureSet_Utf8Validation.VERIFY: + return "VERIFY"; + case FeatureSet_Utf8Validation.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export enum FeatureSet_MessageEncoding { + MESSAGE_ENCODING_UNKNOWN = 0, + LENGTH_PREFIXED = 1, + DELIMITED = 2, + UNRECOGNIZED = -1, +} + +export function featureSet_MessageEncodingFromJSON(object: any): FeatureSet_MessageEncoding { + switch (object) { + case 0: + case "MESSAGE_ENCODING_UNKNOWN": + return FeatureSet_MessageEncoding.MESSAGE_ENCODING_UNKNOWN; + case 1: + case "LENGTH_PREFIXED": + return FeatureSet_MessageEncoding.LENGTH_PREFIXED; + case 2: + case "DELIMITED": + return FeatureSet_MessageEncoding.DELIMITED; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_MessageEncoding.UNRECOGNIZED; + } +} + +export function featureSet_MessageEncodingToJSON(object: FeatureSet_MessageEncoding): string { + switch (object) { + case FeatureSet_MessageEncoding.MESSAGE_ENCODING_UNKNOWN: + return "MESSAGE_ENCODING_UNKNOWN"; + case FeatureSet_MessageEncoding.LENGTH_PREFIXED: + return "LENGTH_PREFIXED"; + case FeatureSet_MessageEncoding.DELIMITED: + return "DELIMITED"; + case FeatureSet_MessageEncoding.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export enum FeatureSet_JsonFormat { + JSON_FORMAT_UNKNOWN = 0, + ALLOW = 1, + LEGACY_BEST_EFFORT = 2, + UNRECOGNIZED = -1, +} + +export function featureSet_JsonFormatFromJSON(object: any): FeatureSet_JsonFormat { + switch (object) { + case 0: + case "JSON_FORMAT_UNKNOWN": + return FeatureSet_JsonFormat.JSON_FORMAT_UNKNOWN; + case 1: + case "ALLOW": + return FeatureSet_JsonFormat.ALLOW; + case 2: + case "LEGACY_BEST_EFFORT": + return FeatureSet_JsonFormat.LEGACY_BEST_EFFORT; + case -1: + case "UNRECOGNIZED": + default: + return FeatureSet_JsonFormat.UNRECOGNIZED; + } +} + +export function featureSet_JsonFormatToJSON(object: FeatureSet_JsonFormat): string { + switch (object) { + case FeatureSet_JsonFormat.JSON_FORMAT_UNKNOWN: + return "JSON_FORMAT_UNKNOWN"; + case FeatureSet_JsonFormat.ALLOW: + return "ALLOW"; + case FeatureSet_JsonFormat.LEGACY_BEST_EFFORT: + return "LEGACY_BEST_EFFORT"; + case FeatureSet_JsonFormat.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +/** + * A compiled specification for the defaults of a set of features. These + * messages are generated from FeatureSet extensions and can be used to seed + * feature resolution. The resolution with this object becomes a simple search + * for the closest matching edition, followed by proto merges. + */ +export interface FeatureSetDefaults { + defaults: FeatureSetDefaults_FeatureSetEditionDefault[]; + /** + * The minimum supported edition (inclusive) when this was constructed. + * Editions before this will not have defaults. + */ + minimumEdition: Edition; + /** + * The maximum known edition (inclusive) when this was constructed. Editions + * after this will not have reliable defaults. + */ + maximumEdition: Edition; +} + +/** + * A map from every known edition with a unique set of defaults to its + * defaults. Not all editions may be contained here. For a given edition, + * the defaults at the closest matching edition ordered at or before it should + * be used. This field must be in strict ascending order by edition. + */ +export interface FeatureSetDefaults_FeatureSetEditionDefault { + edition: Edition; + features: FeatureSet | undefined; +} + /** * Encapsulates information about the original source file from which a * FileDescriptorProto was generated. @@ -1421,7 +1942,7 @@ function createBaseFileDescriptorProto(): FileDescriptorProto { options: undefined, sourceCodeInfo: undefined, syntax: "", - edition: "", + edition: 0, }; } @@ -1467,8 +1988,8 @@ export const FileDescriptorProto = { if (message.syntax !== "") { writer.uint32(98).string(message.syntax); } - if (message.edition !== "") { - writer.uint32(106).string(message.edition); + if (message.edition !== 0) { + writer.uint32(112).int32(message.edition); } return writer; }, @@ -1584,12 +2105,12 @@ export const FileDescriptorProto = { message.syntax = reader.string(); continue; - case 13: - if (tag !== 106) { + case 14: + if (tag !== 112) { break; } - message.edition = reader.string(); + message.edition = reader.int32() as any; continue; } if ((tag & 7) === 4 || tag === 0) { @@ -1620,7 +2141,7 @@ export const FileDescriptorProto = { options: isSet(object.options) ? FileOptions.fromJSON(object.options) : undefined, sourceCodeInfo: isSet(object.sourceCodeInfo) ? SourceCodeInfo.fromJSON(object.sourceCodeInfo) : undefined, syntax: isSet(object.syntax) ? String(object.syntax) : "", - edition: isSet(object.edition) ? String(object.edition) : "", + edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, }; }, @@ -1662,8 +2183,8 @@ export const FileDescriptorProto = { if (message.syntax !== "") { obj.syntax = message.syntax; } - if (message.edition !== "") { - obj.edition = message.edition; + if (message.edition !== 0) { + obj.edition = editionToJSON(message.edition); } return obj; }, @@ -1689,7 +2210,7 @@ export const FileDescriptorProto = { ? SourceCodeInfo.fromPartial(object.sourceCodeInfo) : undefined; message.syntax = object.syntax ?? ""; - message.edition = object.edition ?? ""; + message.edition = object.edition ?? 0; return message; }, }; @@ -2078,7 +2599,7 @@ export const DescriptorProto_ReservedRange = { }; function createBaseExtensionRangeOptions(): ExtensionRangeOptions { - return { uninterpretedOption: [] }; + return { uninterpretedOption: [], declaration: [], features: undefined, verification: 0 }; } export const ExtensionRangeOptions = { @@ -2086,6 +2607,15 @@ export const ExtensionRangeOptions = { for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } + for (const v of message.declaration) { + ExtensionRangeOptions_Declaration.encode(v!, writer.uint32(18).fork()).ldelim(); + } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(402).fork()).ldelim(); + } + if (message.verification !== 0) { + writer.uint32(24).int32(message.verification); + } return writer; }, @@ -2103,6 +2633,27 @@ export const ExtensionRangeOptions = { message.uninterpretedOption.push(UninterpretedOption.decode(reader, reader.uint32())); continue; + case 2: + if (tag !== 18) { + break; + } + + message.declaration.push(ExtensionRangeOptions_Declaration.decode(reader, reader.uint32())); + continue; + case 50: + if (tag !== 402) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.verification = reader.int32() as any; + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -2117,6 +2668,13 @@ export const ExtensionRangeOptions = { uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], + declaration: Array.isArray(object?.declaration) + ? object.declaration.map((e: any) => ExtensionRangeOptions_Declaration.fromJSON(e)) + : [], + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, + verification: isSet(object.verification) + ? extensionRangeOptions_VerificationStateFromJSON(object.verification) + : 0, }; }, @@ -2125,6 +2683,15 @@ export const ExtensionRangeOptions = { if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } + if (message.declaration?.length) { + obj.declaration = message.declaration.map((e) => ExtensionRangeOptions_Declaration.toJSON(e)); + } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } + if (message.verification !== 0) { + obj.verification = extensionRangeOptions_VerificationStateToJSON(message.verification); + } return obj; }, @@ -2134,20 +2701,148 @@ export const ExtensionRangeOptions = { fromPartial, I>>(object: I): ExtensionRangeOptions { const message = createBaseExtensionRangeOptions(); message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; + message.declaration = object.declaration?.map((e) => ExtensionRangeOptions_Declaration.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.verification = object.verification ?? 0; return message; }, }; -function createBaseFieldDescriptorProto(): FieldDescriptorProto { - return { - name: "", - number: 0, - label: 1, - type: 1, - typeName: "", - extendee: "", - defaultValue: "", - oneofIndex: 0, +function createBaseExtensionRangeOptions_Declaration(): ExtensionRangeOptions_Declaration { + return { number: 0, fullName: "", type: "", reserved: false, repeated: false }; +} + +export const ExtensionRangeOptions_Declaration = { + encode(message: ExtensionRangeOptions_Declaration, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.number !== 0) { + writer.uint32(8).int32(message.number); + } + if (message.fullName !== "") { + writer.uint32(18).string(message.fullName); + } + if (message.type !== "") { + writer.uint32(26).string(message.type); + } + if (message.reserved === true) { + writer.uint32(40).bool(message.reserved); + } + if (message.repeated === true) { + writer.uint32(48).bool(message.repeated); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ExtensionRangeOptions_Declaration { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseExtensionRangeOptions_Declaration(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.number = reader.int32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.fullName = reader.string(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.type = reader.string(); + continue; + case 5: + if (tag !== 40) { + break; + } + + message.reserved = reader.bool(); + continue; + case 6: + if (tag !== 48) { + break; + } + + message.repeated = reader.bool(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ExtensionRangeOptions_Declaration { + return { + number: isSet(object.number) ? Number(object.number) : 0, + fullName: isSet(object.fullName) ? String(object.fullName) : "", + type: isSet(object.type) ? String(object.type) : "", + reserved: isSet(object.reserved) ? Boolean(object.reserved) : false, + repeated: isSet(object.repeated) ? Boolean(object.repeated) : false, + }; + }, + + toJSON(message: ExtensionRangeOptions_Declaration): unknown { + const obj: any = {}; + if (message.number !== 0) { + obj.number = Math.round(message.number); + } + if (message.fullName !== "") { + obj.fullName = message.fullName; + } + if (message.type !== "") { + obj.type = message.type; + } + if (message.reserved === true) { + obj.reserved = message.reserved; + } + if (message.repeated === true) { + obj.repeated = message.repeated; + } + return obj; + }, + + create, I>>( + base?: I, + ): ExtensionRangeOptions_Declaration { + return ExtensionRangeOptions_Declaration.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): ExtensionRangeOptions_Declaration { + const message = createBaseExtensionRangeOptions_Declaration(); + message.number = object.number ?? 0; + message.fullName = object.fullName ?? ""; + message.type = object.type ?? ""; + message.reserved = object.reserved ?? false; + message.repeated = object.repeated ?? false; + return message; + }, +}; + +function createBaseFieldDescriptorProto(): FieldDescriptorProto { + return { + name: "", + number: 0, + label: 1, + type: 1, + typeName: "", + extendee: "", + defaultValue: "", + oneofIndex: 0, jsonName: "", options: undefined, proto3Optional: false, @@ -2983,6 +3678,7 @@ function createBaseFileOptions(): FileOptions { phpNamespace: "", phpMetadataNamespace: "", rubyPackage: "", + features: undefined, uninterpretedOption: [], }; } @@ -3049,6 +3745,9 @@ export const FileOptions = { if (message.rubyPackage !== "") { writer.uint32(362).string(message.rubyPackage); } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(402).fork()).ldelim(); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3202,6 +3901,13 @@ export const FileOptions = { message.rubyPackage = reader.string(); continue; + case 50: + if (tag !== 402) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 999: if (tag !== 7994) { break; @@ -3242,6 +3948,7 @@ export const FileOptions = { phpNamespace: isSet(object.phpNamespace) ? String(object.phpNamespace) : "", phpMetadataNamespace: isSet(object.phpMetadataNamespace) ? String(object.phpMetadataNamespace) : "", rubyPackage: isSet(object.rubyPackage) ? String(object.rubyPackage) : "", + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3310,6 +4017,9 @@ export const FileOptions = { if (message.rubyPackage !== "") { obj.rubyPackage = message.rubyPackage; } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -3341,6 +4051,9 @@ export const FileOptions = { message.phpNamespace = object.phpNamespace ?? ""; message.phpMetadataNamespace = object.phpMetadataNamespace ?? ""; message.rubyPackage = object.rubyPackage ?? ""; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -3353,6 +4066,7 @@ function createBaseMessageOptions(): MessageOptions { deprecated: false, mapEntry: false, deprecatedLegacyJsonFieldConflicts: false, + features: undefined, uninterpretedOption: [], }; } @@ -3374,6 +4088,9 @@ export const MessageOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { writer.uint32(88).bool(message.deprecatedLegacyJsonFieldConflicts); } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(98).fork()).ldelim(); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3422,6 +4139,13 @@ export const MessageOptions = { message.deprecatedLegacyJsonFieldConflicts = reader.bool(); continue; + case 12: + if (tag !== 98) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 999: if (tag !== 7994) { break; @@ -3449,6 +4173,7 @@ export const MessageOptions = { deprecatedLegacyJsonFieldConflicts: isSet(object.deprecatedLegacyJsonFieldConflicts) ? Boolean(object.deprecatedLegacyJsonFieldConflicts) : false, + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3472,6 +4197,9 @@ export const MessageOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -3488,6 +4216,9 @@ export const MessageOptions = { message.deprecated = object.deprecated ?? false; message.mapEntry = object.mapEntry ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -3504,7 +4235,9 @@ function createBaseFieldOptions(): FieldOptions { weak: false, debugRedact: false, retention: 0, - target: 0, + targets: [], + editionDefaults: [], + features: undefined, uninterpretedOption: [], }; } @@ -3538,8 +4271,16 @@ export const FieldOptions = { if (message.retention !== 0) { writer.uint32(136).int32(message.retention); } - if (message.target !== 0) { - writer.uint32(144).int32(message.target); + writer.uint32(154).fork(); + for (const v of message.targets) { + writer.int32(v); + } + writer.ldelim(); + for (const v of message.editionDefaults) { + FieldOptions_EditionDefault.encode(v!, writer.uint32(162).fork()).ldelim(); + } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(170).fork()).ldelim(); } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); @@ -3617,12 +4358,36 @@ export const FieldOptions = { message.retention = reader.int32() as any; continue; - case 18: - if (tag !== 144) { + case 19: + if (tag === 152) { + message.targets.push(reader.int32() as any); + + continue; + } + + if (tag === 154) { + const end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) { + message.targets.push(reader.int32() as any); + } + + continue; + } + + break; + case 20: + if (tag !== 162) { break; } - message.target = reader.int32() as any; + message.editionDefaults.push(FieldOptions_EditionDefault.decode(reader, reader.uint32())); + continue; + case 21: + if (tag !== 170) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); continue; case 999: if (tag !== 7994) { @@ -3651,7 +4416,13 @@ export const FieldOptions = { weak: isSet(object.weak) ? Boolean(object.weak) : false, debugRedact: isSet(object.debugRedact) ? Boolean(object.debugRedact) : false, retention: isSet(object.retention) ? fieldOptions_OptionRetentionFromJSON(object.retention) : 0, - target: isSet(object.target) ? fieldOptions_OptionTargetTypeFromJSON(object.target) : 0, + targets: Array.isArray(object?.targets) + ? object.targets.map((e: any) => fieldOptions_OptionTargetTypeFromJSON(e)) + : [], + editionDefaults: Array.isArray(object?.editionDefaults) + ? object.editionDefaults.map((e: any) => FieldOptions_EditionDefault.fromJSON(e)) + : [], + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3687,8 +4458,14 @@ export const FieldOptions = { if (message.retention !== 0) { obj.retention = fieldOptions_OptionRetentionToJSON(message.retention); } - if (message.target !== 0) { - obj.target = fieldOptions_OptionTargetTypeToJSON(message.target); + if (message.targets?.length) { + obj.targets = message.targets.map((e) => fieldOptions_OptionTargetTypeToJSON(e)); + } + if (message.editionDefaults?.length) { + obj.editionDefaults = message.editionDefaults.map((e) => FieldOptions_EditionDefault.toJSON(e)); + } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); @@ -3710,18 +4487,99 @@ export const FieldOptions = { message.weak = object.weak ?? false; message.debugRedact = object.debugRedact ?? false; message.retention = object.retention ?? 0; - message.target = object.target ?? 0; + message.targets = object.targets?.map((e) => e) || []; + message.editionDefaults = object.editionDefaults?.map((e) => FieldOptions_EditionDefault.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; +function createBaseFieldOptions_EditionDefault(): FieldOptions_EditionDefault { + return { edition: 0, value: "" }; +} + +export const FieldOptions_EditionDefault = { + encode(message: FieldOptions_EditionDefault, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.edition !== 0) { + writer.uint32(24).int32(message.edition); + } + if (message.value !== "") { + writer.uint32(18).string(message.value); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): FieldOptions_EditionDefault { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseFieldOptions_EditionDefault(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 3: + if (tag !== 24) { + break; + } + + message.edition = reader.int32() as any; + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): FieldOptions_EditionDefault { + return { + edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, + value: isSet(object.value) ? String(object.value) : "", + }; + }, + + toJSON(message: FieldOptions_EditionDefault): unknown { + const obj: any = {}; + if (message.edition !== 0) { + obj.edition = editionToJSON(message.edition); + } + if (message.value !== "") { + obj.value = message.value; + } + return obj; + }, + + create, I>>(base?: I): FieldOptions_EditionDefault { + return FieldOptions_EditionDefault.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): FieldOptions_EditionDefault { + const message = createBaseFieldOptions_EditionDefault(); + message.edition = object.edition ?? 0; + message.value = object.value ?? ""; + return message; + }, +}; + function createBaseOneofOptions(): OneofOptions { - return { uninterpretedOption: [] }; + return { features: undefined, uninterpretedOption: [] }; } export const OneofOptions = { encode(message: OneofOptions, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(10).fork()).ldelim(); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3735,6 +4593,13 @@ export const OneofOptions = { while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 999: if (tag !== 7994) { break; @@ -3753,6 +4618,7 @@ export const OneofOptions = { fromJSON(object: any): OneofOptions { return { + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3761,6 +4627,9 @@ export const OneofOptions = { toJSON(message: OneofOptions): unknown { const obj: any = {}; + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -3772,13 +4641,22 @@ export const OneofOptions = { }, fromPartial, I>>(object: I): OneofOptions { const message = createBaseOneofOptions(); + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseEnumOptions(): EnumOptions { - return { allowAlias: false, deprecated: false, deprecatedLegacyJsonFieldConflicts: false, uninterpretedOption: [] }; + return { + allowAlias: false, + deprecated: false, + deprecatedLegacyJsonFieldConflicts: false, + features: undefined, + uninterpretedOption: [], + }; } export const EnumOptions = { @@ -3792,6 +4670,9 @@ export const EnumOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { writer.uint32(48).bool(message.deprecatedLegacyJsonFieldConflicts); } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(58).fork()).ldelim(); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3826,6 +4707,13 @@ export const EnumOptions = { message.deprecatedLegacyJsonFieldConflicts = reader.bool(); continue; + case 7: + if (tag !== 58) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 999: if (tag !== 7994) { break; @@ -3849,6 +4737,7 @@ export const EnumOptions = { deprecatedLegacyJsonFieldConflicts: isSet(object.deprecatedLegacyJsonFieldConflicts) ? Boolean(object.deprecatedLegacyJsonFieldConflicts) : false, + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3866,6 +4755,9 @@ export const EnumOptions = { if (message.deprecatedLegacyJsonFieldConflicts === true) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -3880,13 +4772,16 @@ export const EnumOptions = { message.allowAlias = object.allowAlias ?? false; message.deprecated = object.deprecated ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseEnumValueOptions(): EnumValueOptions { - return { deprecated: false, uninterpretedOption: [] }; + return { deprecated: false, features: undefined, debugRedact: false, uninterpretedOption: [] }; } export const EnumValueOptions = { @@ -3894,6 +4789,12 @@ export const EnumValueOptions = { if (message.deprecated === true) { writer.uint32(8).bool(message.deprecated); } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(18).fork()).ldelim(); + } + if (message.debugRedact === true) { + writer.uint32(24).bool(message.debugRedact); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -3914,6 +4815,20 @@ export const EnumValueOptions = { message.deprecated = reader.bool(); continue; + case 2: + if (tag !== 18) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.debugRedact = reader.bool(); + continue; case 999: if (tag !== 7994) { break; @@ -3933,6 +4848,8 @@ export const EnumValueOptions = { fromJSON(object: any): EnumValueOptions { return { deprecated: isSet(object.deprecated) ? Boolean(object.deprecated) : false, + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, + debugRedact: isSet(object.debugRedact) ? Boolean(object.debugRedact) : false, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -3944,6 +4861,12 @@ export const EnumValueOptions = { if (message.deprecated === true) { obj.deprecated = message.deprecated; } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } + if (message.debugRedact === true) { + obj.debugRedact = message.debugRedact; + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -3956,17 +4879,24 @@ export const EnumValueOptions = { fromPartial, I>>(object: I): EnumValueOptions { const message = createBaseEnumValueOptions(); message.deprecated = object.deprecated ?? false; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.debugRedact = object.debugRedact ?? false; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; function createBaseServiceOptions(): ServiceOptions { - return { deprecated: false, uninterpretedOption: [] }; + return { features: undefined, deprecated: false, uninterpretedOption: [] }; } export const ServiceOptions = { encode(message: ServiceOptions, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(274).fork()).ldelim(); + } if (message.deprecated === true) { writer.uint32(264).bool(message.deprecated); } @@ -3983,6 +4913,13 @@ export const ServiceOptions = { while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { + case 34: + if (tag !== 274) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 33: if (tag !== 264) { break; @@ -4008,6 +4945,7 @@ export const ServiceOptions = { fromJSON(object: any): ServiceOptions { return { + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, deprecated: isSet(object.deprecated) ? Boolean(object.deprecated) : false, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) @@ -4017,6 +4955,9 @@ export const ServiceOptions = { toJSON(message: ServiceOptions): unknown { const obj: any = {}; + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.deprecated === true) { obj.deprecated = message.deprecated; } @@ -4031,6 +4972,9 @@ export const ServiceOptions = { }, fromPartial, I>>(object: I): ServiceOptions { const message = createBaseServiceOptions(); + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.deprecated = object.deprecated ?? false; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; @@ -4038,7 +4982,7 @@ export const ServiceOptions = { }; function createBaseMethodOptions(): MethodOptions { - return { deprecated: false, idempotencyLevel: 0, uninterpretedOption: [] }; + return { deprecated: false, idempotencyLevel: 0, features: undefined, uninterpretedOption: [] }; } export const MethodOptions = { @@ -4049,6 +4993,9 @@ export const MethodOptions = { if (message.idempotencyLevel !== 0) { writer.uint32(272).int32(message.idempotencyLevel); } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(282).fork()).ldelim(); + } for (const v of message.uninterpretedOption) { UninterpretedOption.encode(v!, writer.uint32(7994).fork()).ldelim(); } @@ -4076,6 +5023,13 @@ export const MethodOptions = { message.idempotencyLevel = reader.int32() as any; continue; + case 35: + if (tag !== 282) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; case 999: if (tag !== 7994) { break; @@ -4098,6 +5052,7 @@ export const MethodOptions = { idempotencyLevel: isSet(object.idempotencyLevel) ? methodOptions_IdempotencyLevelFromJSON(object.idempotencyLevel) : 0, + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, uninterpretedOption: Array.isArray(object?.uninterpretedOption) ? object.uninterpretedOption.map((e: any) => UninterpretedOption.fromJSON(e)) : [], @@ -4112,6 +5067,9 @@ export const MethodOptions = { if (message.idempotencyLevel !== 0) { obj.idempotencyLevel = methodOptions_IdempotencyLevelToJSON(message.idempotencyLevel); } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } if (message.uninterpretedOption?.length) { obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } @@ -4125,6 +5083,9 @@ export const MethodOptions = { const message = createBaseMethodOptions(); message.deprecated = object.deprecated ?? false; message.idempotencyLevel = object.idempotencyLevel ?? 0; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, @@ -4361,6 +5322,320 @@ export const UninterpretedOption_NamePart = { }, }; +function createBaseFeatureSet(): FeatureSet { + return { + fieldPresence: 0, + enumType: 0, + repeatedFieldEncoding: 0, + utf8Validation: 0, + messageEncoding: 0, + jsonFormat: 0, + }; +} + +export const FeatureSet = { + encode(message: FeatureSet, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.fieldPresence !== 0) { + writer.uint32(8).int32(message.fieldPresence); + } + if (message.enumType !== 0) { + writer.uint32(16).int32(message.enumType); + } + if (message.repeatedFieldEncoding !== 0) { + writer.uint32(24).int32(message.repeatedFieldEncoding); + } + if (message.utf8Validation !== 0) { + writer.uint32(32).int32(message.utf8Validation); + } + if (message.messageEncoding !== 0) { + writer.uint32(40).int32(message.messageEncoding); + } + if (message.jsonFormat !== 0) { + writer.uint32(48).int32(message.jsonFormat); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSet { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseFeatureSet(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.fieldPresence = reader.int32() as any; + continue; + case 2: + if (tag !== 16) { + break; + } + + message.enumType = reader.int32() as any; + continue; + case 3: + if (tag !== 24) { + break; + } + + message.repeatedFieldEncoding = reader.int32() as any; + continue; + case 4: + if (tag !== 32) { + break; + } + + message.utf8Validation = reader.int32() as any; + continue; + case 5: + if (tag !== 40) { + break; + } + + message.messageEncoding = reader.int32() as any; + continue; + case 6: + if (tag !== 48) { + break; + } + + message.jsonFormat = reader.int32() as any; + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): FeatureSet { + return { + fieldPresence: isSet(object.fieldPresence) ? featureSet_FieldPresenceFromJSON(object.fieldPresence) : 0, + enumType: isSet(object.enumType) ? featureSet_EnumTypeFromJSON(object.enumType) : 0, + repeatedFieldEncoding: isSet(object.repeatedFieldEncoding) + ? featureSet_RepeatedFieldEncodingFromJSON(object.repeatedFieldEncoding) + : 0, + utf8Validation: isSet(object.utf8Validation) ? featureSet_Utf8ValidationFromJSON(object.utf8Validation) : 0, + messageEncoding: isSet(object.messageEncoding) ? featureSet_MessageEncodingFromJSON(object.messageEncoding) : 0, + jsonFormat: isSet(object.jsonFormat) ? featureSet_JsonFormatFromJSON(object.jsonFormat) : 0, + }; + }, + + toJSON(message: FeatureSet): unknown { + const obj: any = {}; + if (message.fieldPresence !== 0) { + obj.fieldPresence = featureSet_FieldPresenceToJSON(message.fieldPresence); + } + if (message.enumType !== 0) { + obj.enumType = featureSet_EnumTypeToJSON(message.enumType); + } + if (message.repeatedFieldEncoding !== 0) { + obj.repeatedFieldEncoding = featureSet_RepeatedFieldEncodingToJSON(message.repeatedFieldEncoding); + } + if (message.utf8Validation !== 0) { + obj.utf8Validation = featureSet_Utf8ValidationToJSON(message.utf8Validation); + } + if (message.messageEncoding !== 0) { + obj.messageEncoding = featureSet_MessageEncodingToJSON(message.messageEncoding); + } + if (message.jsonFormat !== 0) { + obj.jsonFormat = featureSet_JsonFormatToJSON(message.jsonFormat); + } + return obj; + }, + + create, I>>(base?: I): FeatureSet { + return FeatureSet.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): FeatureSet { + const message = createBaseFeatureSet(); + message.fieldPresence = object.fieldPresence ?? 0; + message.enumType = object.enumType ?? 0; + message.repeatedFieldEncoding = object.repeatedFieldEncoding ?? 0; + message.utf8Validation = object.utf8Validation ?? 0; + message.messageEncoding = object.messageEncoding ?? 0; + message.jsonFormat = object.jsonFormat ?? 0; + return message; + }, +}; + +function createBaseFeatureSetDefaults(): FeatureSetDefaults { + return { defaults: [], minimumEdition: 0, maximumEdition: 0 }; +} + +export const FeatureSetDefaults = { + encode(message: FeatureSetDefaults, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + for (const v of message.defaults) { + FeatureSetDefaults_FeatureSetEditionDefault.encode(v!, writer.uint32(10).fork()).ldelim(); + } + if (message.minimumEdition !== 0) { + writer.uint32(32).int32(message.minimumEdition); + } + if (message.maximumEdition !== 0) { + writer.uint32(40).int32(message.maximumEdition); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSetDefaults { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseFeatureSetDefaults(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.defaults.push(FeatureSetDefaults_FeatureSetEditionDefault.decode(reader, reader.uint32())); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.minimumEdition = reader.int32() as any; + continue; + case 5: + if (tag !== 40) { + break; + } + + message.maximumEdition = reader.int32() as any; + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): FeatureSetDefaults { + return { + defaults: Array.isArray(object?.defaults) + ? object.defaults.map((e: any) => FeatureSetDefaults_FeatureSetEditionDefault.fromJSON(e)) + : [], + minimumEdition: isSet(object.minimumEdition) ? editionFromJSON(object.minimumEdition) : 0, + maximumEdition: isSet(object.maximumEdition) ? editionFromJSON(object.maximumEdition) : 0, + }; + }, + + toJSON(message: FeatureSetDefaults): unknown { + const obj: any = {}; + if (message.defaults?.length) { + obj.defaults = message.defaults.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.toJSON(e)); + } + if (message.minimumEdition !== 0) { + obj.minimumEdition = editionToJSON(message.minimumEdition); + } + if (message.maximumEdition !== 0) { + obj.maximumEdition = editionToJSON(message.maximumEdition); + } + return obj; + }, + + create, I>>(base?: I): FeatureSetDefaults { + return FeatureSetDefaults.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): FeatureSetDefaults { + const message = createBaseFeatureSetDefaults(); + message.defaults = object.defaults?.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(e)) || []; + message.minimumEdition = object.minimumEdition ?? 0; + message.maximumEdition = object.maximumEdition ?? 0; + return message; + }, +}; + +function createBaseFeatureSetDefaults_FeatureSetEditionDefault(): FeatureSetDefaults_FeatureSetEditionDefault { + return { edition: 0, features: undefined }; +} + +export const FeatureSetDefaults_FeatureSetEditionDefault = { + encode(message: FeatureSetDefaults_FeatureSetEditionDefault, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.edition !== 0) { + writer.uint32(24).int32(message.edition); + } + if (message.features !== undefined) { + FeatureSet.encode(message.features, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): FeatureSetDefaults_FeatureSetEditionDefault { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseFeatureSetDefaults_FeatureSetEditionDefault(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 3: + if (tag !== 24) { + break; + } + + message.edition = reader.int32() as any; + continue; + case 2: + if (tag !== 18) { + break; + } + + message.features = FeatureSet.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): FeatureSetDefaults_FeatureSetEditionDefault { + return { + edition: isSet(object.edition) ? editionFromJSON(object.edition) : 0, + features: isSet(object.features) ? FeatureSet.fromJSON(object.features) : undefined, + }; + }, + + toJSON(message: FeatureSetDefaults_FeatureSetEditionDefault): unknown { + const obj: any = {}; + if (message.edition !== 0) { + obj.edition = editionToJSON(message.edition); + } + if (message.features !== undefined) { + obj.features = FeatureSet.toJSON(message.features); + } + return obj; + }, + + create, I>>( + base?: I, + ): FeatureSetDefaults_FeatureSetEditionDefault { + return FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): FeatureSetDefaults_FeatureSetEditionDefault { + const message = createBaseFeatureSetDefaults_FeatureSetEditionDefault(); + message.edition = object.edition ?? 0; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + return message; + }, +}; + function createBaseSourceCodeInfo(): SourceCodeInfo { return { location: [] }; } diff --git a/ui/grpc_generated/google/protobuf/timestamp.ts b/ui/grpc_generated/google/protobuf/timestamp.ts index 560af8a4e..959778faa 100644 --- a/ui/grpc_generated/google/protobuf/timestamp.ts +++ b/ui/grpc_generated/google/protobuf/timestamp.ts @@ -92,7 +92,7 @@ export const protobufPackage = "google.protobuf"; * [`strftime`](https://docs.python.org/2/library/time.html#time.strftime) with * the time format spec '%Y-%m-%dT%H:%M:%S.%fZ'. Likewise, in Java, one can use * the Joda Time's [`ISODateTimeFormat.dateTime()`]( - * http://www.joda.org/joda-time/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime%2D%2D + * http://joda-time.sourceforge.net/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime() * ) to obtain a formatter capable of generating timestamps in this format. */ export interface Timestamp { From 70f1030400c5000e1a173c068b06548df2b21ae5 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Fri, 24 Nov 2023 21:01:48 +0530 Subject: [PATCH 8/8] change to 10 mins --- flow/activities/flowable.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index d7900a9b5..ddeba8ea2 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -699,7 +699,7 @@ func getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { } func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { - sendTimeout := 10 * time.Second + sendTimeout := 10 * time.Minute ticker := time.NewTicker(sendTimeout) defer ticker.Stop()