diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 31ef2a2989..1b9ced3a3c 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -10,6 +10,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + "github.com/sirupsen/logrus" "google.golang.org/protobuf/proto" ) @@ -146,36 +147,36 @@ func (h *FlowRequestHandler) GetColumns( defer peerPool.Close() rows, err := peerPool.Query(ctx, ` - SELECT + SELECT cols.column_name, cols.data_type, - CASE + CASE WHEN constraint_type = 'PRIMARY KEY' THEN true ELSE false END AS is_primary_key - FROM + FROM information_schema.columns cols - LEFT JOIN + LEFT JOIN ( - SELECT + SELECT kcu.column_name, tc.constraint_type - FROM + FROM information_schema.key_column_usage kcu - JOIN + JOIN information_schema.table_constraints tc - ON + ON kcu.constraint_name = tc.constraint_name AND kcu.constraint_schema = tc.constraint_schema AND kcu.constraint_name = tc.constraint_name - WHERE + WHERE tc.constraint_type = 'PRIMARY KEY' AND kcu.table_schema = $1 AND kcu.table_name = $2 ) AS pk - ON + ON cols.column_name = pk.column_name - WHERE + WHERE cols.table_schema = $3 AND cols.table_name = $4; `, req.SchemaName, req.TableName, req.SchemaName, req.TableName) @@ -210,14 +211,17 @@ func (h *FlowRequestHandler) GetSlotInfo( pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) if err != nil { + logrus.Errorf("Failed to create postgres connector: %v", err) return &protos.PeerSlotResponse{SlotData: nil}, err } defer pgConnector.Close() slotInfo, err := pgConnector.GetSlotInfo("") if err != nil { + logrus.Errorf("Failed to get slot info: %v", err) return &protos.PeerSlotResponse{SlotData: nil}, err } + return &protos.PeerSlotResponse{ SlotData: slotInfo, }, nil @@ -227,16 +231,27 @@ func (h *FlowRequestHandler) GetStatInfo( ctx context.Context, req *protos.PostgresPeerActivityInfoRequest, ) (*protos.PeerStatResponse, error) { - peerPool, peerUser, err := h.getPoolForPGPeer(ctx, req.PeerName) + pgConfig, err := h.getPGPeerConfig(ctx, req.PeerName) if err != nil { return &protos.PeerStatResponse{StatData: nil}, err } - defer peerPool.Close() + + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + if err != nil { + logrus.Errorf("Failed to create postgres connector: %v", err) + return &protos.PeerStatResponse{StatData: nil}, err + } + defer pgConnector.Close() + + peerPool := pgConnector.GetPool() + peerUser := pgConfig.User + rows, err := peerPool.Query(ctx, "SELECT pid, wait_event, wait_event_type, query_start::text, query,"+ "EXTRACT(epoch FROM(now()-query_start)) AS dur"+ " FROM pg_stat_activity WHERE "+ "usename=$1 AND state != 'idle';", peerUser) if err != nil { + logrus.Errorf("Failed to get stat info: %v", err) return &protos.PeerStatResponse{StatData: nil}, err } defer rows.Close() @@ -251,6 +266,7 @@ func (h *FlowRequestHandler) GetStatInfo( err := rows.Scan(&pid, &waitEvent, &waitEventType, &queryStart, &query, &duration) if err != nil { + logrus.Errorf("Failed to scan row: %v", err) return &protos.PeerStatResponse{StatData: nil}, err } @@ -288,6 +304,7 @@ func (h *FlowRequestHandler) GetStatInfo( Duration: float32(d), }) } + return &protos.PeerStatResponse{ StatData: statInfoRows, }, nil diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index eb59f1ab8d..2bf7fb7d04 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -25,8 +25,8 @@ type PostgresConnector struct { connStr string ctx context.Context config *protos.PostgresConfig - pool *pgxpool.Pool - replPool *pgxpool.Pool + pool *SSHWrappedPostgresPool + replPool *SSHWrappedPostgresPool tableSchemaMapping map[string]*protos.TableSchema customTypesMapping map[uint32]string metadataSchema string @@ -51,12 +51,12 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) // set pool size to 3 to avoid connection pool exhaustion connConfig.MaxConns = 3 - pool, err := pgxpool.NewWithConfig(ctx, connConfig) + pool, err := NewSSHWrappedPostgresPool(ctx, connConfig, pgConfig.SshConfig) if err != nil { return nil, fmt.Errorf("failed to create connection pool: %w", err) } - customTypeMap, err := utils.GetCustomDataTypes(ctx, pool) + customTypeMap, err := utils.GetCustomDataTypes(ctx, pool.Pool) if err != nil { return nil, fmt.Errorf("failed to get custom type map: %w", err) } @@ -73,7 +73,7 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) // TODO: replPool not initializing might be intentional, if we only want to use QRep mirrors // and the user doesn't have the REPLICATION permission - replPool, err := pgxpool.NewWithConfig(ctx, replConnConfig) + replPool, err := NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) if err != nil { return nil, fmt.Errorf("failed to create connection pool: %w", err) } @@ -94,6 +94,11 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) }, nil } +// GetPool returns the connection pool. +func (c *PostgresConnector) GetPool() *SSHWrappedPostgresPool { + return c.pool +} + // Close closes all connections. func (c *PostgresConnector) Close() error { if c.pool != nil { @@ -230,7 +235,7 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) error { cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ AppContext: c.ctx, - Connection: c.replPool, + Connection: c.replPool.Pool, SrcTableIDNameMapping: req.SrcTableIDNameMapping, Slot: slotName, Publication: publicationName, diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 6d514dccf4..2c7fa9b295 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -315,7 +315,8 @@ func (c *PostgresConnector) PullQRepRecords( log.WithFields(log.Fields{ "partitionId": partition.PartitionId, }).Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -361,7 +362,8 @@ func (c *PostgresConnector) PullQRepRecords( return nil, err } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -386,7 +388,8 @@ func (c *PostgresConnector) PullQRepRecordStream( "flowName": config.FlowJobName, "partitionId": partition.PartitionId, }).Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -434,7 +437,8 @@ func (c *PostgresConnector) PullQRepRecordStream( return 0, err } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -558,7 +562,8 @@ func (c *PostgresConnector) PullXminRecordStream( query += " WHERE age(xmin) > 0 AND age(xmin) <= age($1::xid)" } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, currentSnapshotXmin, err diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 9eead5ec34..bc6956feec 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -70,9 +70,9 @@ func TestGetQRepPartitions(t *testing.T) { t.Fatalf("Failed to parse config: %v", err) } - pool, err := pgxpool.NewWithConfig(context.Background(), config) + pool, err := NewSSHWrappedPostgresPool(context.Background(), config, nil) if err != nil { - t.Fatalf("unable to connect to database: %v", err) + t.Fatalf("Failed to create pool: %v", err) } // Generate a random schema name @@ -101,7 +101,7 @@ func TestGetQRepPartitions(t *testing.T) { } // from 2010 Jan 1 10:00 AM UTC to 2010 Jan 30 10:00 AM UTC - numRows := prepareTestData(t, pool, schemaName) + numRows := prepareTestData(t, pool.Pool, schemaName) secondsInADay := uint32(24 * time.Hour / time.Second) fmt.Printf("secondsInADay: %d\n", secondsInADay) diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go new file mode 100644 index 0000000000..ef19f78b32 --- /dev/null +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -0,0 +1,166 @@ +package connpostgres + +import ( + "context" + "fmt" + "net" + "sync" + "time" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/sirupsen/logrus" + "golang.org/x/crypto/ssh" +) + +type SSHWrappedPostgresPool struct { + *pgxpool.Pool + + poolConfig *pgxpool.Config + sshConfig *ssh.ClientConfig + sshServer string + once sync.Once + sshClient *ssh.Client + ctx context.Context + cancel context.CancelFunc +} + +func NewSSHWrappedPostgresPool( + ctx context.Context, + poolConfig *pgxpool.Config, + sshConfig *protos.SSHConfig, +) (*SSHWrappedPostgresPool, error) { + swCtx, cancel := context.WithCancel(ctx) + + var sshServer string + var clientConfig *ssh.ClientConfig + + if sshConfig != nil { + sshServer = fmt.Sprintf("%s:%d", sshConfig.Host, sshConfig.Port) + var err error + clientConfig, err = utils.GetSSHClientConfig( + sshConfig.User, + sshConfig.Password, + sshConfig.PrivateKey, + ) + if err != nil { + logrus.Error("Failed to get SSH client config: ", err) + cancel() + return nil, err + } + } + + pool := &SSHWrappedPostgresPool{ + poolConfig: poolConfig, + sshConfig: clientConfig, + sshServer: sshServer, + ctx: swCtx, + cancel: cancel, + } + + err := pool.connect() + if err != nil { + return nil, err + } + + return pool, nil +} + +func (swpp *SSHWrappedPostgresPool) connect() error { + var err error + swpp.once.Do(func() { + err = swpp.setupSSH() + if err != nil { + return + } + + swpp.Pool, err = pgxpool.NewWithConfig(swpp.ctx, swpp.poolConfig) + if err != nil { + logrus.Errorf("Failed to create pool: %v", err) + return + } + + logrus.Infof("Established pool to %s:%d", + swpp.poolConfig.ConnConfig.Host, swpp.poolConfig.ConnConfig.Port) + + err = retryWithBackoff(func() error { + err = swpp.Ping(swpp.ctx) + if err != nil { + logrus.Errorf("Failed to ping pool: %v", err) + return err + } + return nil + }, 5, 5*time.Second) + + if err != nil { + logrus.Errorf("Failed to create pool: %v", err) + } + }) + + if err == nil { + logrus.Info("Successfully connected to Postgres") + } + + return err +} + +func (swpp *SSHWrappedPostgresPool) setupSSH() error { + if swpp.sshConfig == nil { + logrus.Info("SSH config is nil, skipping SSH setup") + return nil + } + + logrus.Info("Setting up SSH connection to ", swpp.sshServer) + + var err error + swpp.sshClient, err = ssh.Dial("tcp", swpp.sshServer, swpp.sshConfig) + if err != nil { + return err + } + + swpp.poolConfig.ConnConfig.DialFunc = func(ctx context.Context, network, addr string) (net.Conn, error) { + conn, err := swpp.sshClient.Dial(network, addr) + if err != nil { + return nil, err + } + return &noDeadlineConn{Conn: conn}, nil + } + + return nil +} + +func (swpp *SSHWrappedPostgresPool) Close() { + swpp.cancel() + + if swpp.Pool != nil { + swpp.Pool.Close() + } + + if swpp.sshClient != nil { + swpp.sshClient.Close() + } +} + +type retryFunc func() error + +func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) (err error) { + for i := 0; i < maxRetries; i++ { + err = fn() + if err == nil { + return nil + } + if i < maxRetries-1 { + logrus.Infof("Attempt #%d failed, retrying in %s", i+1, backoff) + time.Sleep(backoff) + } + } + return err +} + +// see: https://github.com/jackc/pgx/issues/382#issuecomment-1496586216 +type noDeadlineConn struct{ net.Conn } + +func (c *noDeadlineConn) SetDeadline(t time.Time) error { return nil } +func (c *noDeadlineConn) SetReadDeadline(t time.Time) error { return nil } +func (c *noDeadlineConn) SetWriteDeadline(t time.Time) error { return nil } diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go new file mode 100644 index 0000000000..7bd8ed141f --- /dev/null +++ b/flow/connectors/utils/ssh.go @@ -0,0 +1,48 @@ +package utils + +import ( + "encoding/base64" + "fmt" + + "golang.org/x/crypto/ssh" +) + +// getSSHClientConfig returns an *ssh.ClientConfig based on provided credentials. +// Parameters: +// +// user: SSH username +// password: SSH password (can be empty if using a private key) +// privateKeyString: Private key as a string (can be empty if using a password) +func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientConfig, error) { + var authMethods []ssh.AuthMethod + + // Password-based authentication + if password != "" { + authMethods = append(authMethods, ssh.Password(password)) + } + + // Private key-based authentication + if privateKeyString != "" { + pkey, err := base64.StdEncoding.DecodeString(privateKeyString) + if err != nil { + return nil, fmt.Errorf("failed to base64 decode private key: %w", err) + } + + signer, err := ssh.ParsePrivateKey(pkey) + if err != nil { + return nil, fmt.Errorf("failed to parse private key: %w", err) + } + + authMethods = append(authMethods, ssh.PublicKeys(signer)) + } + + if len(authMethods) == 0 { + return nil, fmt.Errorf("no authentication methods provided") + } + + return &ssh.ClientConfig{ + User: user, + Auth: authMethods, + HostKeyCallback: ssh.InsecureIgnoreHostKey(), + }, nil +} diff --git a/flow/generated/protos/peers.pb.go b/flow/generated/protos/peers.pb.go index 6ac88a4c11..aa03dcfd82 100644 --- a/flow/generated/protos/peers.pb.go +++ b/flow/generated/protos/peers.pb.go @@ -84,6 +84,85 @@ func (DBType) EnumDescriptor() ([]byte, []int) { return file_peers_proto_rawDescGZIP(), []int{0} } +type SSHConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Host string `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"` + Port uint32 `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"` + User string `protobuf:"bytes,3,opt,name=user,proto3" json:"user,omitempty"` + Password string `protobuf:"bytes,4,opt,name=password,proto3" json:"password,omitempty"` + PrivateKey string `protobuf:"bytes,5,opt,name=private_key,json=privateKey,proto3" json:"private_key,omitempty"` +} + +func (x *SSHConfig) Reset() { + *x = SSHConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_peers_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SSHConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SSHConfig) ProtoMessage() {} + +func (x *SSHConfig) ProtoReflect() protoreflect.Message { + mi := &file_peers_proto_msgTypes[0] + 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 SSHConfig.ProtoReflect.Descriptor instead. +func (*SSHConfig) Descriptor() ([]byte, []int) { + return file_peers_proto_rawDescGZIP(), []int{0} +} + +func (x *SSHConfig) GetHost() string { + if x != nil { + return x.Host + } + return "" +} + +func (x *SSHConfig) GetPort() uint32 { + if x != nil { + return x.Port + } + return 0 +} + +func (x *SSHConfig) GetUser() string { + if x != nil { + return x.User + } + return "" +} + +func (x *SSHConfig) GetPassword() string { + if x != nil { + return x.Password + } + return "" +} + +func (x *SSHConfig) GetPrivateKey() string { + if x != nil { + return x.PrivateKey + } + return "" +} + type SnowflakeConfig struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -105,7 +184,7 @@ type SnowflakeConfig struct { func (x *SnowflakeConfig) Reset() { *x = SnowflakeConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[0] + mi := &file_peers_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -118,7 +197,7 @@ func (x *SnowflakeConfig) String() string { func (*SnowflakeConfig) ProtoMessage() {} func (x *SnowflakeConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[0] + mi := &file_peers_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -131,7 +210,7 @@ func (x *SnowflakeConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use SnowflakeConfig.ProtoReflect.Descriptor instead. func (*SnowflakeConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{0} + return file_peers_proto_rawDescGZIP(), []int{1} } func (x *SnowflakeConfig) GetAccountId() string { @@ -225,7 +304,7 @@ type BigqueryConfig struct { func (x *BigqueryConfig) Reset() { *x = BigqueryConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[1] + mi := &file_peers_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -238,7 +317,7 @@ func (x *BigqueryConfig) String() string { func (*BigqueryConfig) ProtoMessage() {} func (x *BigqueryConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[1] + mi := &file_peers_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -251,7 +330,7 @@ func (x *BigqueryConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use BigqueryConfig.ProtoReflect.Descriptor instead. func (*BigqueryConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{1} + return file_peers_proto_rawDescGZIP(), []int{2} } func (x *BigqueryConfig) GetAuthType() string { @@ -346,7 +425,7 @@ type MongoConfig struct { func (x *MongoConfig) Reset() { *x = MongoConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[2] + mi := &file_peers_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -359,7 +438,7 @@ func (x *MongoConfig) String() string { func (*MongoConfig) ProtoMessage() {} func (x *MongoConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[2] + mi := &file_peers_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -372,7 +451,7 @@ func (x *MongoConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use MongoConfig.ProtoReflect.Descriptor instead. func (*MongoConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{2} + return file_peers_proto_rawDescGZIP(), []int{3} } func (x *MongoConfig) GetUsername() string { @@ -423,13 +502,14 @@ type PostgresConfig struct { // this is used only in query replication mode right now. TransactionSnapshot string `protobuf:"bytes,6,opt,name=transaction_snapshot,json=transactionSnapshot,proto3" json:"transaction_snapshot,omitempty"` // defaults to _peerdb_internal - MetadataSchema *string `protobuf:"bytes,7,opt,name=metadata_schema,json=metadataSchema,proto3,oneof" json:"metadata_schema,omitempty"` + MetadataSchema *string `protobuf:"bytes,7,opt,name=metadata_schema,json=metadataSchema,proto3,oneof" json:"metadata_schema,omitempty"` + SshConfig *SSHConfig `protobuf:"bytes,8,opt,name=ssh_config,json=sshConfig,proto3,oneof" json:"ssh_config,omitempty"` } func (x *PostgresConfig) Reset() { *x = PostgresConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[3] + mi := &file_peers_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -442,7 +522,7 @@ func (x *PostgresConfig) String() string { func (*PostgresConfig) ProtoMessage() {} func (x *PostgresConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[3] + mi := &file_peers_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -455,7 +535,7 @@ func (x *PostgresConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use PostgresConfig.ProtoReflect.Descriptor instead. func (*PostgresConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{3} + return file_peers_proto_rawDescGZIP(), []int{4} } func (x *PostgresConfig) GetHost() string { @@ -507,6 +587,13 @@ func (x *PostgresConfig) GetMetadataSchema() string { return "" } +func (x *PostgresConfig) GetSshConfig() *SSHConfig { + if x != nil { + return x.SshConfig + } + return nil +} + type EventHubConfig struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -527,7 +614,7 @@ type EventHubConfig struct { func (x *EventHubConfig) Reset() { *x = EventHubConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -540,7 +627,7 @@ func (x *EventHubConfig) String() string { func (*EventHubConfig) ProtoMessage() {} func (x *EventHubConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -553,7 +640,7 @@ func (x *EventHubConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use EventHubConfig.ProtoReflect.Descriptor instead. func (*EventHubConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{4} + return file_peers_proto_rawDescGZIP(), []int{5} } func (x *EventHubConfig) GetNamespace() string { @@ -619,7 +706,7 @@ type EventHubGroupConfig struct { func (x *EventHubGroupConfig) Reset() { *x = EventHubGroupConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[5] + mi := &file_peers_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -632,7 +719,7 @@ func (x *EventHubGroupConfig) String() string { func (*EventHubGroupConfig) ProtoMessage() {} func (x *EventHubGroupConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[5] + mi := &file_peers_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -645,7 +732,7 @@ func (x *EventHubGroupConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use EventHubGroupConfig.ProtoReflect.Descriptor instead. func (*EventHubGroupConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{5} + return file_peers_proto_rawDescGZIP(), []int{6} } func (x *EventHubGroupConfig) GetEventhubs() map[string]*EventHubConfig { @@ -686,7 +773,7 @@ type S3Config struct { func (x *S3Config) Reset() { *x = S3Config{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[6] + mi := &file_peers_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -699,7 +786,7 @@ func (x *S3Config) String() string { func (*S3Config) ProtoMessage() {} func (x *S3Config) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[6] + mi := &file_peers_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -712,7 +799,7 @@ func (x *S3Config) ProtoReflect() protoreflect.Message { // Deprecated: Use S3Config.ProtoReflect.Descriptor instead. func (*S3Config) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{6} + return file_peers_proto_rawDescGZIP(), []int{7} } func (x *S3Config) GetUrl() string { @@ -779,7 +866,7 @@ type SqlServerConfig struct { func (x *SqlServerConfig) Reset() { *x = SqlServerConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[7] + mi := &file_peers_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -792,7 +879,7 @@ func (x *SqlServerConfig) String() string { func (*SqlServerConfig) ProtoMessage() {} func (x *SqlServerConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[7] + mi := &file_peers_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -805,7 +892,7 @@ func (x *SqlServerConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use SqlServerConfig.ProtoReflect.Descriptor instead. func (*SqlServerConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{7} + return file_peers_proto_rawDescGZIP(), []int{8} } func (x *SqlServerConfig) GetServer() string { @@ -866,7 +953,7 @@ type Peer struct { func (x *Peer) Reset() { *x = Peer{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[8] + mi := &file_peers_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -879,7 +966,7 @@ func (x *Peer) String() string { func (*Peer) ProtoMessage() {} func (x *Peer) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[8] + mi := &file_peers_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -892,7 +979,7 @@ func (x *Peer) ProtoReflect() protoreflect.Message { // Deprecated: Use Peer.ProtoReflect.Descriptor instead. func (*Peer) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{8} + return file_peers_proto_rawDescGZIP(), []int{9} } func (x *Peer) GetName() string { @@ -1028,209 +1115,223 @@ var File_peers_proto protoreflect.FileDescriptor var file_peers_proto_rawDesc = []byte{ 0x0a, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x22, 0xf7, 0x02, 0x0a, 0x0f, - 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, - 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, - 0x6f, 0x75, 0x73, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x77, 0x61, 0x72, 0x65, - 0x68, 0x6f, 0x75, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, - 0x0a, 0x0e, 0x73, 0x33, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x33, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, - 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, - 0x6f, 0x72, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x48, - 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, - 0x64, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x99, 0x03, 0x0a, 0x0e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1b, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x75, 0x74, - 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, - 0x63, 0x74, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, - 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, - 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x12, 0x1b, - 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x61, - 0x75, 0x74, 0x68, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, - 0x75, 0x74, 0x68, 0x55, 0x72, 0x69, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x5f, - 0x75, 0x72, 0x69, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x6b, 0x65, 0x6e, - 0x55, 0x72, 0x69, 0x12, 0x3c, 0x0a, 0x1b, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x70, 0x72, 0x6f, 0x76, - 0x69, 0x64, 0x65, 0x72, 0x5f, 0x78, 0x35, 0x30, 0x39, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, - 0x72, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x61, 0x75, 0x74, 0x68, 0x50, 0x72, - 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x58, 0x35, 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, - 0x6c, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x78, 0x35, 0x30, 0x39, - 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x11, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x58, 0x35, 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, - 0x72, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x49, - 0x64, 0x22, 0xa3, 0x01, 0x0a, 0x0b, 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, - 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, - 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xf9, 0x01, 0x0a, 0x0e, 0x50, 0x6f, 0x73, 0x74, - 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, - 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, - 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, - 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, - 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, - 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x31, - 0x0a, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6e, - 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x72, - 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, - 0x74, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0e, 0x6d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x42, - 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1a, 0x0a, 0x08, 0x6c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x09, + 0x53, 0x53, 0x48, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, + 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x22, 0xf7, 0x02, 0x0a, 0x0f, 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, + 0x0a, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, 0x6f, 0x75, 0x73, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, 0x6f, 0x75, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, + 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x33, 0x5f, 0x69, 0x6e, 0x74, 0x65, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, + 0x33, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x08, + 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, + 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, + 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, + 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x99, 0x03, 0x0a, + 0x0e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x1b, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x61, 0x75, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, + 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x70, + 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x49, + 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x6d, 0x61, + 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x75, 0x74, 0x68, 0x55, 0x72, 0x69, 0x12, 0x1b, 0x0a, + 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x55, 0x72, 0x69, 0x12, 0x3c, 0x0a, 0x1b, 0x61, 0x75, + 0x74, 0x68, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x78, 0x35, 0x30, 0x39, + 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x17, 0x61, 0x75, 0x74, 0x68, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x58, 0x35, 0x30, + 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x78, 0x35, 0x30, 0x39, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x58, 0x35, + 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, + 0x61, 0x73, 0x65, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, + 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x49, 0x64, 0x22, 0xa3, 0x01, 0x0a, 0x0b, 0x4d, 0x6f, 0x6e, + 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, + 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, + 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, + 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xc5, + 0x02, 0x0a, 0x0e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, + 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, + 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x48, 0x00, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x3b, 0x0a, 0x0a, 0x73, 0x73, 0x68, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x53, 0x48, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x48, 0x01, 0x52, 0x09, 0x73, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x88, 0x01, 0x01, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x73, 0x73, 0x68, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1a, + 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x04, 0x20, 0x01, 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, 0x0a, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x39, 0x0a, 0x19, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x79, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x44, 0x61, 0x79, 0x73, 0x22, 0xa7, 0x02, 0x0a, 0x13, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4e, + 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x30, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x12, 0x3d, + 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x02, 0x20, + 0x01, 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, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x25, 0x0a, + 0x0e, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x5a, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xe0, 0x02, 0x0a, 0x08, 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, + 0x27, 0x0a, 0x0d, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x4b, 0x65, 0x79, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08, 0x72, 0x6f, 0x6c, + 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x07, 0x72, + 0x6f, 0x6c, 0x65, 0x41, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x06, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1f, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x07, 0x20, 0x01, 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, 0x0a, 0x6d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, - 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, - 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x39, 0x0a, 0x19, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, - 0x5f, 0x64, 0x61, 0x79, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x44, - 0x61, 0x79, 0x73, 0x22, 0xa7, 0x02, 0x0a, 0x13, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4e, 0x0a, 0x09, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x02, 0x20, 0x01, 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, 0x0a, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x25, 0x0a, 0x0e, 0x75, 0x6e, - 0x6e, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0d, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x1a, 0x5a, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x73, 0x65, 0x63, + 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x42, 0x0b, + 0x0a, 0x09, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, + 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, + 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, + 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, + 0x6f, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, + 0x6f, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, + 0x91, 0x05, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, + 0x61, 0x6b, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, + 0x00, 0x52, 0x0f, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, 0x69, 0x67, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, 0x0c, 0x6d, + 0x6f, 0x6e, 0x67, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0b, + 0x6d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x70, + 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe0, 0x02, - 0x0a, 0x08, 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, - 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x27, 0x0a, 0x0d, - 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, - 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, - 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x48, 0x01, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, - 0x4b, 0x65, 0x79, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, - 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x07, 0x72, 0x6f, 0x6c, 0x65, - 0x41, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, - 0x88, 0x01, 0x01, 0x12, 0x1f, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x5f, 0x64, 0x62, 0x18, 0x07, 0x20, 0x01, 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, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x44, 0x62, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, - 0x65, 0x79, 0x5f, 0x69, 0x64, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, - 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x42, 0x0b, 0x0a, 0x09, 0x5f, - 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x72, 0x65, 0x67, - 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, - 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, - 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0x91, 0x05, 0x0a, - 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x6e, 0x6f, - 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0f, - 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, 0x0c, 0x6d, 0x6f, 0x6e, 0x67, - 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x4d, 0x6f, - 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x6f, 0x6e, - 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x70, 0x6f, 0x73, 0x74, - 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x06, 0x20, 0x01, 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, 0x48, - 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, - 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x68, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x35, 0x0a, 0x09, 0x73, 0x33, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x33, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x08, 0x73, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x71, 0x6c, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x71, - 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x57, 0x0a, - 0x15, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, - 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, + 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x35, 0x0a, + 0x09, 0x73, 0x33, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x08, 0x73, 0x33, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x71, + 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, + 0x0f, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x57, 0x0a, 0x15, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 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, } var ( @@ -1246,40 +1347,42 @@ 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 - (*BigqueryConfig)(nil), // 2: peerdb_peers.BigqueryConfig - (*MongoConfig)(nil), // 3: peerdb_peers.MongoConfig - (*PostgresConfig)(nil), // 4: peerdb_peers.PostgresConfig - (*EventHubConfig)(nil), // 5: peerdb_peers.EventHubConfig - (*EventHubGroupConfig)(nil), // 6: peerdb_peers.EventHubGroupConfig - (*S3Config)(nil), // 7: peerdb_peers.S3Config - (*SqlServerConfig)(nil), // 8: peerdb_peers.SqlServerConfig - (*Peer)(nil), // 9: peerdb_peers.Peer - nil, // 10: peerdb_peers.EventHubGroupConfig.EventhubsEntry + (*SSHConfig)(nil), // 1: peerdb_peers.SSHConfig + (*SnowflakeConfig)(nil), // 2: peerdb_peers.SnowflakeConfig + (*BigqueryConfig)(nil), // 3: peerdb_peers.BigqueryConfig + (*MongoConfig)(nil), // 4: peerdb_peers.MongoConfig + (*PostgresConfig)(nil), // 5: peerdb_peers.PostgresConfig + (*EventHubConfig)(nil), // 6: peerdb_peers.EventHubConfig + (*EventHubGroupConfig)(nil), // 7: peerdb_peers.EventHubGroupConfig + (*S3Config)(nil), // 8: peerdb_peers.S3Config + (*SqlServerConfig)(nil), // 9: peerdb_peers.SqlServerConfig + (*Peer)(nil), // 10: peerdb_peers.Peer + 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 - 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 - 1, // 5: peerdb_peers.Peer.snowflake_config:type_name -> peerdb_peers.SnowflakeConfig - 2, // 6: peerdb_peers.Peer.bigquery_config:type_name -> peerdb_peers.BigqueryConfig - 3, // 7: peerdb_peers.Peer.mongo_config:type_name -> peerdb_peers.MongoConfig - 4, // 8: peerdb_peers.Peer.postgres_config:type_name -> peerdb_peers.PostgresConfig - 5, // 9: peerdb_peers.Peer.eventhub_config:type_name -> peerdb_peers.EventHubConfig - 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 + 1, // 0: peerdb_peers.PostgresConfig.ssh_config:type_name -> peerdb_peers.SSHConfig + 5, // 1: peerdb_peers.EventHubConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig + 11, // 2: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry + 5, // 3: peerdb_peers.EventHubGroupConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig + 5, // 4: peerdb_peers.S3Config.metadata_db:type_name -> peerdb_peers.PostgresConfig + 0, // 5: peerdb_peers.Peer.type:type_name -> peerdb_peers.DBType + 2, // 6: peerdb_peers.Peer.snowflake_config:type_name -> peerdb_peers.SnowflakeConfig + 3, // 7: peerdb_peers.Peer.bigquery_config:type_name -> peerdb_peers.BigqueryConfig + 4, // 8: peerdb_peers.Peer.mongo_config:type_name -> peerdb_peers.MongoConfig + 5, // 9: peerdb_peers.Peer.postgres_config:type_name -> peerdb_peers.PostgresConfig + 6, // 10: peerdb_peers.Peer.eventhub_config:type_name -> peerdb_peers.EventHubConfig + 8, // 11: peerdb_peers.Peer.s3_config:type_name -> peerdb_peers.S3Config + 9, // 12: peerdb_peers.Peer.sqlserver_config:type_name -> peerdb_peers.SqlServerConfig + 7, // 13: peerdb_peers.Peer.eventhub_group_config:type_name -> peerdb_peers.EventHubGroupConfig + 6, // 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() } @@ -1289,7 +1392,7 @@ func file_peers_proto_init() { } if !protoimpl.UnsafeEnabled { file_peers_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SnowflakeConfig); i { + switch v := v.(*SSHConfig); i { case 0: return &v.state case 1: @@ -1301,7 +1404,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BigqueryConfig); i { + switch v := v.(*SnowflakeConfig); i { case 0: return &v.state case 1: @@ -1313,7 +1416,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MongoConfig); i { + switch v := v.(*BigqueryConfig); i { case 0: return &v.state case 1: @@ -1325,7 +1428,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresConfig); i { + switch v := v.(*MongoConfig); i { case 0: return &v.state case 1: @@ -1337,7 +1440,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventHubConfig); i { + switch v := v.(*PostgresConfig); i { case 0: return &v.state case 1: @@ -1349,7 +1452,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventHubGroupConfig); i { + switch v := v.(*EventHubConfig); i { case 0: return &v.state case 1: @@ -1361,7 +1464,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*S3Config); i { + switch v := v.(*EventHubGroupConfig); i { case 0: return &v.state case 1: @@ -1373,7 +1476,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SqlServerConfig); i { + switch v := v.(*S3Config); i { case 0: return &v.state case 1: @@ -1385,6 +1488,18 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SqlServerConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_peers_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Peer); i { case 0: return &v.state @@ -1397,10 +1512,10 @@ func file_peers_proto_init() { } } } - file_peers_proto_msgTypes[0].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[3].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[6].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[8].OneofWrappers = []interface{}{ + file_peers_proto_msgTypes[1].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[4].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[7].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[9].OneofWrappers = []interface{}{ (*Peer_SnowflakeConfig)(nil), (*Peer_BigqueryConfig)(nil), (*Peer_MongoConfig)(nil), @@ -1416,7 +1531,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/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 49784c2ff4..f65fe4b2d0 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -647,6 +647,7 @@ fn parse_db_options( .to_string(), metadata_schema: opts.get("metadata_schema").map(|s| s.to_string()), transaction_snapshot: "".to_string(), + ssh_config: None, }; let config = Config::PostgresConfig(postgres_config); Some(config) diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 77467348e3..94119ff6c1 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -75,6 +75,7 @@ impl CatalogConfig { database: self.database.clone(), transaction_snapshot: "".to_string(), metadata_schema: Some("".to_string()), + ssh_config: None, } } diff --git a/nexus/pt/src/peerdb_peers.rs b/nexus/pt/src/peerdb_peers.rs index 017a36b8ab..8266eea158 100644 --- a/nexus/pt/src/peerdb_peers.rs +++ b/nexus/pt/src/peerdb_peers.rs @@ -1,6 +1,20 @@ // @generated #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SshConfig { + #[prost(string, tag="1")] + pub host: ::prost::alloc::string::String, + #[prost(uint32, tag="2")] + pub port: u32, + #[prost(string, tag="3")] + pub user: ::prost::alloc::string::String, + #[prost(string, tag="4")] + pub password: ::prost::alloc::string::String, + #[prost(string, tag="5")] + pub private_key: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct SnowflakeConfig { #[prost(string, tag="1")] pub account_id: ::prost::alloc::string::String, @@ -83,6 +97,8 @@ pub struct PostgresConfig { /// defaults to _peerdb_internal #[prost(string, optional, tag="7")] pub metadata_schema: ::core::option::Option<::prost::alloc::string::String>, + #[prost(message, optional, tag="8")] + pub ssh_config: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_peers.serde.rs b/nexus/pt/src/peerdb_peers.serde.rs index 2cdf144f4b..18c206865e 100644 --- a/nexus/pt/src/peerdb_peers.serde.rs +++ b/nexus/pt/src/peerdb_peers.serde.rs @@ -1134,6 +1134,9 @@ impl serde::Serialize for PostgresConfig { if self.metadata_schema.is_some() { len += 1; } + if self.ssh_config.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_peers.PostgresConfig", len)?; if !self.host.is_empty() { struct_ser.serialize_field("host", &self.host)?; @@ -1156,6 +1159,9 @@ impl serde::Serialize for PostgresConfig { if let Some(v) = self.metadata_schema.as_ref() { struct_ser.serialize_field("metadataSchema", v)?; } + if let Some(v) = self.ssh_config.as_ref() { + struct_ser.serialize_field("sshConfig", v)?; + } struct_ser.end() } } @@ -1175,6 +1181,8 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { "transactionSnapshot", "metadata_schema", "metadataSchema", + "ssh_config", + "sshConfig", ]; #[allow(clippy::enum_variant_names)] @@ -1186,6 +1194,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { Database, TransactionSnapshot, MetadataSchema, + SshConfig, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1215,6 +1224,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { "database" => Ok(GeneratedField::Database), "transactionSnapshot" | "transaction_snapshot" => Ok(GeneratedField::TransactionSnapshot), "metadataSchema" | "metadata_schema" => Ok(GeneratedField::MetadataSchema), + "sshConfig" | "ssh_config" => Ok(GeneratedField::SshConfig), _ => Ok(GeneratedField::__SkipField__), } } @@ -1241,6 +1251,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { let mut database__ = None; let mut transaction_snapshot__ = None; let mut metadata_schema__ = None; + let mut ssh_config__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Host => { @@ -1287,6 +1298,12 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { } metadata_schema__ = map.next_value()?; } + GeneratedField::SshConfig => { + if ssh_config__.is_some() { + return Err(serde::de::Error::duplicate_field("sshConfig")); + } + ssh_config__ = map.next_value()?; + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1300,6 +1317,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { database: database__.unwrap_or_default(), transaction_snapshot: transaction_snapshot__.unwrap_or_default(), metadata_schema: metadata_schema__, + ssh_config: ssh_config__, }) } } @@ -1507,6 +1525,172 @@ impl<'de> serde::Deserialize<'de> for S3Config { deserializer.deserialize_struct("peerdb_peers.S3Config", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SshConfig { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.host.is_empty() { + len += 1; + } + if self.port != 0 { + len += 1; + } + if !self.user.is_empty() { + len += 1; + } + if !self.password.is_empty() { + len += 1; + } + if !self.private_key.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_peers.SSHConfig", len)?; + if !self.host.is_empty() { + struct_ser.serialize_field("host", &self.host)?; + } + if self.port != 0 { + struct_ser.serialize_field("port", &self.port)?; + } + if !self.user.is_empty() { + struct_ser.serialize_field("user", &self.user)?; + } + if !self.password.is_empty() { + struct_ser.serialize_field("password", &self.password)?; + } + if !self.private_key.is_empty() { + struct_ser.serialize_field("privateKey", &self.private_key)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SshConfig { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "host", + "port", + "user", + "password", + "private_key", + "privateKey", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Host, + Port, + User, + Password, + PrivateKey, + __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 { + "host" => Ok(GeneratedField::Host), + "port" => Ok(GeneratedField::Port), + "user" => Ok(GeneratedField::User), + "password" => Ok(GeneratedField::Password), + "privateKey" | "private_key" => Ok(GeneratedField::PrivateKey), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SshConfig; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_peers.SSHConfig") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut host__ = None; + let mut port__ = None; + let mut user__ = None; + let mut password__ = None; + let mut private_key__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Host => { + if host__.is_some() { + return Err(serde::de::Error::duplicate_field("host")); + } + host__ = Some(map.next_value()?); + } + GeneratedField::Port => { + if port__.is_some() { + return Err(serde::de::Error::duplicate_field("port")); + } + port__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::User => { + if user__.is_some() { + return Err(serde::de::Error::duplicate_field("user")); + } + user__ = Some(map.next_value()?); + } + GeneratedField::Password => { + if password__.is_some() { + return Err(serde::de::Error::duplicate_field("password")); + } + password__ = Some(map.next_value()?); + } + GeneratedField::PrivateKey => { + if private_key__.is_some() { + return Err(serde::de::Error::duplicate_field("privateKey")); + } + private_key__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(SshConfig { + host: host__.unwrap_or_default(), + port: port__.unwrap_or_default(), + user: user__.unwrap_or_default(), + password: password__.unwrap_or_default(), + private_key: private_key__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_peers.SSHConfig", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SnowflakeConfig { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/peers.proto b/protos/peers.proto index fe343d906e..18a4577aeb 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -2,6 +2,14 @@ syntax = "proto3"; package peerdb_peers; +message SSHConfig { + string host = 1; + uint32 port = 2; + string user = 3; + string password = 4; + string private_key = 5; +} + message SnowflakeConfig { string account_id = 1; string username = 2; @@ -48,6 +56,7 @@ message PostgresConfig { string transaction_snapshot = 6; // defaults to _peerdb_internal optional string metadata_schema = 7; + optional SSHConfig ssh_config = 8; } message EventHubConfig { diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index ca74ba0127..8580014956 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -65,6 +65,7 @@ const constructPeer = ( export async function POST(request: Request) { const body = await request.json(); + console.log('POST Validate Peer:', body); const { name, type, config, mode } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); const peer = constructPeer(name, type, config); diff --git a/ui/app/dto/PeersDTO.ts b/ui/app/dto/PeersDTO.ts index 0e6dfef818..232e1b2c3c 100644 --- a/ui/app/dto/PeersDTO.ts +++ b/ui/app/dto/PeersDTO.ts @@ -43,3 +43,4 @@ export type CatalogPeer = { type: number; options: Buffer; }; +export type PeerSetter = React.Dispatch>; diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index 62b2bb26c2..b1c27e0edc 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -1,5 +1,4 @@ -import { PeerConfig } from '@/app/dto/PeersDTO'; -import { PeerSetter } from '@/components/ConfigForm'; +import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from './bq'; import { blankPostgresSetting } from './pg'; import { blankS3Setting } from './s3'; diff --git a/ui/app/peers/create/[peerType]/helpers/pg.ts b/ui/app/peers/create/[peerType]/helpers/pg.ts index 84c464bf7a..8e822fb5c4 100644 --- a/ui/app/peers/create/[peerType]/helpers/pg.ts +++ b/ui/app/peers/create/[peerType]/helpers/pg.ts @@ -1,4 +1,5 @@ -import { PostgresConfig } from '@/grpc_generated/peers'; +import { PostgresConfig, SSHConfig } from '@/grpc_generated/peers'; +import { Dispatch, SetStateAction } from 'react'; import { PeerSetting } from './common'; export const postgresSetting: PeerSetting[] = [ @@ -48,6 +49,53 @@ export const postgresSetting: PeerSetting[] = [ }, ]; +type sshSetter = Dispatch>; +export const sshSetting = [ + { + label: 'Host', + stateHandler: (value: string, setter: sshSetter) => + setter((curr: SSHConfig) => ({ ...curr, host: value })), + tips: 'Specifies the IP host name or address of your instance.', + }, + { + label: 'Port', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, port: parseInt(value, 10) })), + type: 'number', + default: 5432, + tips: 'Specifies the TCP/IP port or local Unix domain socket file extension on which clients can connect.', + }, + { + label: 'User', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, user: value })), + tips: 'Specify the user that we should use to connect to this host.', + }, + { + label: 'Password', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, password: value })), + type: 'password', + optional: true, + tips: 'Password associated with the user you provided.', + }, + { + label: 'BASE64 Private Key', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, privateKey: value })), + optional: true, + tips: 'Private key as a BASE64 string for authentication in order to SSH into your machine.', + }, +]; + +export const blankSSHConfig: SSHConfig = { + host: '', + port: 22, + user: '', + password: '', + privateKey: '', +}; + export const blankPostgresSetting: PostgresConfig = { host: '', port: 5432, diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index cbefbfad8b..a611df87ab 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -1,7 +1,9 @@ 'use client'; import { PeerConfig } from '@/app/dto/PeersDTO'; -import BQConfig from '@/components/BigqueryConfig'; -import S3ConfigForm from '@/components/S3Form'; +import BigqueryForm from '@/components/PeerForms/BigqueryConfig'; +import PostgresForm from '@/components/PeerForms/PostgresForm'; +import S3Form from '@/components/PeerForms/S3Form'; +import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; import { Button } from '@/lib/Button'; import { ButtonGroup } from '@/lib/ButtonGroup'; import { Label } from '@/lib/Label'; @@ -12,9 +14,8 @@ import { Tooltip } from '@/lib/Tooltip'; import Link from 'next/link'; import { useRouter } from 'next/navigation'; import { useState } from 'react'; -import ConfigForm from '../../../../components/ConfigForm'; import { handleCreate, handleValidate } from './handlers'; -import { PeerSetting, getBlankSetting } from './helpers/common'; +import { getBlankSetting } from './helpers/common'; import { postgresSetting } from './helpers/pg'; import { snowflakeSetting } from './helpers/sf'; @@ -36,18 +37,15 @@ export default function CreateConfig({ }); const [loading, setLoading] = useState(false); const configComponentMap = (dbType: string) => { - const configForm = (settingList: PeerSetting[]) => ( - - ); switch (dbType) { case 'POSTGRES': - return configForm(postgresSetting); + return ; case 'SNOWFLAKE': - return configForm(snowflakeSetting); + return ; case 'BIGQUERY': - return ; + return ; case 'S3': - return ; + return ; default: return <>; } @@ -103,7 +101,7 @@ export default function CreateConfig({ - {dbType && configComponentMap(dbType)} + {configComponentMap(dbType)} diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 01a82386e2..5bedeaa26f 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -52,6 +52,42 @@ export const pgSchema = z.object({ .string() .max(100, 'Transaction snapshot too long (100 char limit)') .optional(), + sshConfig: z + .object({ + host: z + .string({ + required_error: 'SSH Host is required', + invalid_type_error: 'SSH Host must be a string', + }) + .min(1, { message: 'SSH Host cannot be empty' }) + .max(255, 'SSH Host must be less than 255 characters'), + port: z + .number({ + required_error: 'SSH Port is required', + invalid_type_error: 'SSH Port must be a number', + }) + .int() + .min(1, 'SSH Port must be a positive integer') + .max(65535, 'SSH Port must be below 65535'), + user: z + .string({ + required_error: 'SSH User is required', + invalid_type_error: 'SSH User must be a string', + }) + .min(1, 'SSH User must be non-empty') + .max(64, 'SSH User must be less than 64 characters'), + password: z + .string({ + required_error: 'SSH Password is required', + invalid_type_error: 'SSH Password must be a string', + }) + .max(100, 'SSH Password must be less than 100 characters'), + privateKey: z.string({ + required_error: 'SSH Private Key is required', + invalid_type_error: 'SSH Private Key must be a string', + }), + }) + .optional(), }); export const sfSchema = z.object({ diff --git a/ui/components/BigqueryConfig.tsx b/ui/components/PeerForms/BigqueryConfig.tsx similarity index 96% rename from ui/components/BigqueryConfig.tsx rename to ui/components/PeerForms/BigqueryConfig.tsx index 9da7c9601d..db435bb17a 100644 --- a/ui/components/BigqueryConfig.tsx +++ b/ui/components/PeerForms/BigqueryConfig.tsx @@ -1,4 +1,5 @@ 'use client'; +import { PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from '@/app/peers/create/[peerType]/helpers/bq'; import { BigqueryConfig } from '@/grpc_generated/peers'; import { Label } from '@/lib/Label'; @@ -7,13 +8,12 @@ import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; import Link from 'next/link'; import { useState } from 'react'; -import { PeerSetter } from './ConfigForm'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; interface BQProps { setter: PeerSetter; } -export default function BQConfig(props: BQProps) { +export default function BigqueryForm(props: BQProps) { const [datasetID, setDatasetID] = useState(''); const handleJSONFile = (file: File) => { if (file) { diff --git a/ui/components/PeerForms/PostgresForm.tsx b/ui/components/PeerForms/PostgresForm.tsx new file mode 100644 index 0000000000..1819e626df --- /dev/null +++ b/ui/components/PeerForms/PostgresForm.tsx @@ -0,0 +1,156 @@ +'use client'; +import { PeerSetter } from '@/app/dto/PeersDTO'; +import { PeerSetting } from '@/app/peers/create/[peerType]/helpers/common'; +import { + blankSSHConfig, + sshSetting, +} from '@/app/peers/create/[peerType]/helpers/pg'; +import { SSHConfig } from '@/grpc_generated/peers'; +import { Label } from '@/lib/Label'; +import { RowWithTextField } from '@/lib/Layout'; +import { Switch } from '@/lib/Switch'; +import { TextField } from '@/lib/TextField'; +import { Tooltip } from '@/lib/Tooltip'; +import { useEffect, useState } from 'react'; +import { InfoPopover } from '../InfoPopover'; +interface ConfigProps { + settings: PeerSetting[]; + setter: PeerSetter; +} + +export default function PostgresForm({ settings, setter }: ConfigProps) { + const [showSSH, setShowSSH] = useState(false); + const [sshConfig, setSSHConfig] = useState(blankSSHConfig); + + const handleChange = ( + e: React.ChangeEvent, + setting: PeerSetting + ) => { + setting.stateHandler(e.target.value, setter); + }; + + useEffect(() => { + setter((prev) => { + return { + ...prev, + sshConfig: showSSH ? sshConfig : undefined, + }; + }); + }, [sshConfig, setter, showSSH]); + + return ( + <> + {settings.map((setting, id) => { + return ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ ) => + handleChange(e, setting) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ); + })} + + + +
+ + setShowSSH(state)} /> +
+ {showSSH && + sshSetting.map((sshParam, index) => ( + + {sshParam.label}{' '} + {!sshParam.optional && ( + + + + )} + + } + action={ +
+ ) => + sshParam.stateHandler(e.target.value, setSSHConfig) + } + type={sshParam.type} + defaultValue={ + (sshConfig as SSHConfig)[ + sshParam.label === 'BASE64 Private Key' + ? 'privateKey' + : (sshParam.label.toLowerCase() as + | 'host' + | 'port' + | 'user' + | 'password' + | 'privateKey') + ] || '' + } + /> + {sshParam.tips && } +
+ } + /> + ))} + + ); +} diff --git a/ui/components/S3Form.tsx b/ui/components/PeerForms/S3Form.tsx similarity index 93% rename from ui/components/S3Form.tsx rename to ui/components/PeerForms/S3Form.tsx index 2a3f005671..cafef330f9 100644 --- a/ui/components/S3Form.tsx +++ b/ui/components/PeerForms/S3Form.tsx @@ -1,5 +1,5 @@ 'use client'; -import { PeerConfig } from '@/app/dto/PeersDTO'; +import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { postgresSetting } from '@/app/peers/create/[peerType]/helpers/pg'; import { blankS3Setting, @@ -13,13 +13,12 @@ import { Switch } from '@/lib/Switch'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; import { useEffect, useState } from 'react'; -import { PeerSetter } from './ConfigForm'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; interface S3Props { setter: PeerSetter; } -const S3ConfigForm = ({ setter }: S3Props) => { +const S3Form = ({ setter }: S3Props) => { const [showMetadata, setShowMetadata] = useState(false); const [metadataDB, setMetadataDB] = useState( blankS3Setting.metadataDb! @@ -31,6 +30,7 @@ const S3ConfigForm = ({ setter }: S3Props) => { storageType === 'GCS' ); }; + useEffect(() => { const endpoint = storageType === 'S3' ? '' : 'storage.googleapis.com'; setter((prev) => { @@ -189,7 +189,12 @@ const S3ConfigForm = ({ setter }: S3Props) => { } defaultValue={ (metadataDB as PostgresConfig)[ - pgSetting.label.toLowerCase() as keyof PostgresConfig + pgSetting.label.toLowerCase() as + | 'host' + | 'port' + | 'user' + | 'password' + | 'database' ] || '' } /> @@ -208,4 +213,4 @@ const S3ConfigForm = ({ setter }: S3Props) => { ); }; -export default S3ConfigForm; +export default S3Form; diff --git a/ui/components/ConfigForm.tsx b/ui/components/PeerForms/SnowflakeForm.tsx similarity index 91% rename from ui/components/ConfigForm.tsx rename to ui/components/PeerForms/SnowflakeForm.tsx index 7cd816415c..e2db335723 100644 --- a/ui/components/ConfigForm.tsx +++ b/ui/components/PeerForms/SnowflakeForm.tsx @@ -1,19 +1,18 @@ 'use client'; -import { PeerConfig } from '@/app/dto/PeersDTO'; +import { PeerSetter } from '@/app/dto/PeersDTO'; import { PeerSetting } from '@/app/peers/create/[peerType]/helpers/common'; import { Label } from '@/lib/Label'; import { RowWithTextField } from '@/lib/Layout'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; -export type PeerSetter = React.Dispatch>; interface ConfigProps { settings: PeerSetting[]; setter: PeerSetter; } -export default function ConfigForm(props: ConfigProps) { +export default function SnowflakeForm(props: ConfigProps) { const handleFile = ( file: File, setFile: (value: string, setter: PeerSetter) => void diff --git a/ui/grpc_generated/peers.ts b/ui/grpc_generated/peers.ts index 8d4b9b4eef..7cdd2ca50b 100644 --- a/ui/grpc_generated/peers.ts +++ b/ui/grpc_generated/peers.ts @@ -73,6 +73,14 @@ export function dBTypeToJSON(object: DBType): string { } } +export interface SSHConfig { + host: string; + port: number; + user: string; + password: string; + privateKey: string; +} + export interface SnowflakeConfig { accountId: string; username: string; @@ -121,6 +129,7 @@ export interface PostgresConfig { transactionSnapshot: string; /** defaults to _peerdb_internal */ metadataSchema?: string | undefined; + sshConfig?: SSHConfig | undefined; } export interface EventHubConfig { @@ -181,6 +190,125 @@ export interface Peer { eventhubGroupConfig?: EventHubGroupConfig | undefined; } +function createBaseSSHConfig(): SSHConfig { + return { host: "", port: 0, user: "", password: "", privateKey: "" }; +} + +export const SSHConfig = { + encode(message: SSHConfig, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.host !== "") { + writer.uint32(10).string(message.host); + } + if (message.port !== 0) { + writer.uint32(16).uint32(message.port); + } + if (message.user !== "") { + writer.uint32(26).string(message.user); + } + if (message.password !== "") { + writer.uint32(34).string(message.password); + } + if (message.privateKey !== "") { + writer.uint32(42).string(message.privateKey); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SSHConfig { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSSHConfig(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.host = reader.string(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.port = reader.uint32(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.user = reader.string(); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.password = reader.string(); + continue; + case 5: + if (tag !== 42) { + break; + } + + message.privateKey = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SSHConfig { + return { + host: isSet(object.host) ? String(object.host) : "", + port: isSet(object.port) ? Number(object.port) : 0, + user: isSet(object.user) ? String(object.user) : "", + password: isSet(object.password) ? String(object.password) : "", + privateKey: isSet(object.privateKey) ? String(object.privateKey) : "", + }; + }, + + toJSON(message: SSHConfig): unknown { + const obj: any = {}; + if (message.host !== "") { + obj.host = message.host; + } + if (message.port !== 0) { + obj.port = Math.round(message.port); + } + if (message.user !== "") { + obj.user = message.user; + } + if (message.password !== "") { + obj.password = message.password; + } + if (message.privateKey !== "") { + obj.privateKey = message.privateKey; + } + return obj; + }, + + create, I>>(base?: I): SSHConfig { + return SSHConfig.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): SSHConfig { + const message = createBaseSSHConfig(); + message.host = object.host ?? ""; + message.port = object.port ?? 0; + message.user = object.user ?? ""; + message.password = object.password ?? ""; + message.privateKey = object.privateKey ?? ""; + return message; + }, +}; + function createBaseSnowflakeConfig(): SnowflakeConfig { return { accountId: "", @@ -735,6 +863,7 @@ function createBasePostgresConfig(): PostgresConfig { database: "", transactionSnapshot: "", metadataSchema: undefined, + sshConfig: undefined, }; } @@ -761,6 +890,9 @@ export const PostgresConfig = { if (message.metadataSchema !== undefined) { writer.uint32(58).string(message.metadataSchema); } + if (message.sshConfig !== undefined) { + SSHConfig.encode(message.sshConfig, writer.uint32(66).fork()).ldelim(); + } return writer; }, @@ -820,6 +952,13 @@ export const PostgresConfig = { message.metadataSchema = reader.string(); continue; + case 8: + if (tag !== 66) { + break; + } + + message.sshConfig = SSHConfig.decode(reader, reader.uint32()); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -838,6 +977,7 @@ export const PostgresConfig = { database: isSet(object.database) ? String(object.database) : "", transactionSnapshot: isSet(object.transactionSnapshot) ? String(object.transactionSnapshot) : "", metadataSchema: isSet(object.metadataSchema) ? String(object.metadataSchema) : undefined, + sshConfig: isSet(object.sshConfig) ? SSHConfig.fromJSON(object.sshConfig) : undefined, }; }, @@ -864,6 +1004,9 @@ export const PostgresConfig = { if (message.metadataSchema !== undefined) { obj.metadataSchema = message.metadataSchema; } + if (message.sshConfig !== undefined) { + obj.sshConfig = SSHConfig.toJSON(message.sshConfig); + } return obj; }, @@ -879,6 +1022,9 @@ export const PostgresConfig = { message.database = object.database ?? ""; message.transactionSnapshot = object.transactionSnapshot ?? ""; message.metadataSchema = object.metadataSchema ?? undefined; + message.sshConfig = (object.sshConfig !== undefined && object.sshConfig !== null) + ? SSHConfig.fromPartial(object.sshConfig) + : undefined; return message; }, };