From d7fc82847daef0c9ad1cb0b1e3714660a158cb04 Mon Sep 17 00:00:00 2001 From: Alisha Date: Fri, 12 Apr 2024 17:04:58 -0700 Subject: [PATCH 01/12] custom benthos input --- .../sql/input_generate_table_records.go | 233 ++++++++++++ worker/internal/benthos/sql/input_sql_raw.go | 5 + .../benthos/sql/input_sql_select_generate.go | 333 ++++++++++++++++++ .../workflows/datasync/workflow/workflow.go | 2 +- 4 files changed, 572 insertions(+), 1 deletion(-) create mode 100644 worker/internal/benthos/sql/input_generate_table_records.go create mode 100644 worker/internal/benthos/sql/input_sql_select_generate.go diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go new file mode 100644 index 0000000000..dd4259455b --- /dev/null +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -0,0 +1,233 @@ +package neosync_benthos_sql + +// combo of generate, sql select and mapping + +import ( + "context" + "encoding/json" + "fmt" + "sync" + + "github.com/benthosdev/benthos/v4/public/bloblang" + "github.com/benthosdev/benthos/v4/public/component/input" + "github.com/benthosdev/benthos/v4/public/component/interop" + "github.com/benthosdev/benthos/v4/public/service" + mysql_queries "github.com/nucleuscloud/neosync/backend/gen/go/db/dbschemas/mysql" + neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" + "github.com/nucleuscloud/neosync/worker/internal/benthos/shutdown" +) + +func generateTableRecordsInputSpec() *service.ConfigSpec { + return service.NewConfigSpec(). + Field(service.NewStringField("driver")). + Field(service.NewStringField("dsn")). + Field(service.NewStringField("query")). + Field(service.NewAnyMapField("table_columns_map")). + Field(service.NewIntField("count")). + Field(service.NewBloblangField("args_mapping").Optional()) +} +func RegisterGenerateTableRecordsInput(env *service.Environment, dbprovider DbPoolProvider, stopActivityChannel chan error) error { + return env.RegisterBatchInput( + "generate_table_records", generateTableRecordsInputSpec(), + func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { + // input, err := newSqlSelectGenerateInput(conf, mgr, dbprovider, stopActivityChannel) + // if err != nil { + // return nil, err + // } + // return input, nil + nm := interop.UnwrapManagement(mgr) + b, err := newGenerateReaderFromParsed(conf, nm) + if err != nil { + return nil, err + } + i, err := input.NewAsyncReader("generate", input.NewAsyncPreserver(b), nm) + if err != nil { + return nil, err + } + return interop.NewUnwrapInternalInput(i), nil + }, + ) +} + +//------------------------------------------------------------------------------ + +type generateReader struct { + driver string + dsn string + tableColsMap map[string][]string + provider DbPoolProvider + logger *service.Logger + + argsMapping *bloblang.Executor + + db mysql_queries.DBTX + dbMut sync.Mutex + // rows *sql.Rows + remaining int + index int + joinedRows []map[string]any + + shutSig *shutdown.Signaller + + stopActivityChannel chan error +} + +func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resources) (*generateReader, error) { + driver, err := conf.FieldString("driver") + if err != nil { + return nil, err + } + dsn, err := conf.FieldString("dsn") + if err != nil { + return nil, err + } + + count, err := conf.FieldInt("count") + if err != nil { + return nil, err + } + + tmpMap, err := conf.FieldAnyMap("table_columns_map") + if err != nil { + return nil, err + } + tableColsMap := map[string][]string{} + for k, v := range tmpMap { + val, err := v.FieldStringList() + if err != nil { + return nil, err + } + tableColsMap[k] = val + } + var argsMapping *bloblang.Executor + if conf.Contains("args_mapping") { + argsMapping, err = conf.FieldBloblang("args_mapping") + if err != nil { + return nil, err + } + } + + return &generateReader{ + logger: mgr.Logger(), + shutSig: shutdown.NewSignaller(), + driver: driver, + dsn: dsn, + tableColsMap: tableColsMap, + argsMapping: argsMapping, + provider: dbprovider, + stopActivityChannel: channel, + remaining: count, + index: 0, + }, nil +} + +// Connect establishes a Bloblang reader. +func (b *generateReader) Connect(ctx context.Context) error { + s.logger.Debug("connecting to pooled input") + s.dbMut.Lock() + defer s.dbMut.Unlock() + + if s.db != nil { + return nil + } + + db, err := s.provider.GetDb(s.driver, s.dsn) + if err != nil { + return nil + } + + var args []any + if s.argsMapping != nil { + iargs, err := s.argsMapping.Query(nil) + if err != nil { + return err + } + var ok bool + if args, ok = iargs.([]any); !ok { + return fmt.Errorf("mapping returned non-array result: %T", iargs) + } + } + + rows, err := db.QueryContext(ctx, s.queryStatic, args...) + if err != nil { + if !neosync_benthos.IsMaxConnectionError(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return err + } + jsonF, _ := json.MarshalIndent(rows, "", " ") + fmt.Printf("\n rows: %s \n", string(jsonF)) + + s.rows = rows + go func() { + <-s.shutSig.CloseNowChan() + + s.dbMut.Lock() + if s.rows != nil { + _ = s.rows.Close() + s.rows = nil + } + // not closing the connection here as that is managed by an outside force + s.db = nil + s.dbMut.Unlock() + + s.shutSig.ShutdownComplete() + }() + return nil + return nil +} + +// ReadBatch a new bloblang generated message. +func (b *generateReader) ReadBatch(ctx context.Context) (message.Batch, input.AsyncAckFn, error) { + batchSize := b.batchSize + if b.limited { + if b.remaining <= 0 { + return nil, nil, component.ErrTypeClosed + } + if b.remaining < batchSize { + batchSize = b.remaining + } + } + + if !b.firstIsFree && b.timer != nil { + select { + case t, open := <-b.timer.C: + if !open { + return nil, nil, component.ErrTypeClosed + } + if b.schedule != nil { + b.timer.Reset(getDurationTillNextSchedule(t, *b.schedule, b.location)) + } + case <-ctx.Done(): + return nil, nil, component.ErrTimeout + } + } + b.firstIsFree = false + + batch := make(message.Batch, 0, batchSize) + for i := 0; i < batchSize; i++ { + p, err := b.exec.MapPart(0, batch) + if err != nil { + return nil, nil, err + } + if p != nil { + if b.limited { + b.remaining-- + } + batch = append(batch, p) + } + } + if len(batch) == 0 { + return nil, nil, component.ErrTimeout + } + return batch, func(context.Context, error) error { return nil }, nil +} + +// CloseAsync shuts down the bloblang reader. +func (b *generateReader) Close(ctx context.Context) (err error) { + if b.timer != nil { + b.timer.Stop() + } + return +} diff --git a/worker/internal/benthos/sql/input_sql_raw.go b/worker/internal/benthos/sql/input_sql_raw.go index 2e05a79423..ec8553e10b 100644 --- a/worker/internal/benthos/sql/input_sql_raw.go +++ b/worker/internal/benthos/sql/input_sql_raw.go @@ -3,6 +3,7 @@ package neosync_benthos_sql import ( "context" "database/sql" + "encoding/json" "fmt" "sync" @@ -124,6 +125,8 @@ func (s *pooledInput) Connect(ctx context.Context) error { } return err } + jsonF, _ := json.MarshalIndent(rows, "", " ") + fmt.Printf("\n rows: %s \n", string(jsonF)) s.rows = rows go func() { @@ -169,6 +172,8 @@ func (s *pooledInput) Read(ctx context.Context) (*service.Message, service.AckFu s.rows = nil return nil, nil, err } + jsonF, _ := json.MarshalIndent(obj, "", " ") + fmt.Printf("\n obj: %s \n", string(jsonF)) msg := service.NewMessage(nil) msg.SetStructured(obj) diff --git a/worker/internal/benthos/sql/input_sql_select_generate.go b/worker/internal/benthos/sql/input_sql_select_generate.go new file mode 100644 index 0000000000..b3d6a6347a --- /dev/null +++ b/worker/internal/benthos/sql/input_sql_select_generate.go @@ -0,0 +1,333 @@ +package neosync_benthos_sql + +// combo of generate, sql select and mapping + +import ( + "context" + "database/sql" + "fmt" + "sync" + + "github.com/benthosdev/benthos/v4/public/bloblang" + "github.com/benthosdev/benthos/v4/public/service" + "github.com/doug-martin/goqu/v9" + "github.com/doug-martin/goqu/v9/exp" + mysql_queries "github.com/nucleuscloud/neosync/backend/gen/go/db/dbschemas/mysql" + neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" + "github.com/nucleuscloud/neosync/worker/internal/benthos/shutdown" + transformer_utils "github.com/nucleuscloud/neosync/worker/internal/benthos/transformers/utils" +) + +func sqlSelectGenerateInputSpec() *service.ConfigSpec { + return service.NewConfigSpec(). + Field(service.NewStringField("driver")). + Field(service.NewStringField("dsn")). + Field(service.NewStringField("query")). + Field(service.NewAnyMapField("table_columns_map")). + Field(service.NewIntField("count")). + Field(service.NewBloblangField("args_mapping").Optional()) +} + +// Registers an input on a benthos environment called pooled_sql_raw +func RegisterSqlSelectGenerateInput(env *service.Environment, dbprovider DbPoolProvider, stopActivityChannel chan error) error { + return env.RegisterInput( + "pooled_sql_select_generate", sqlSelectGenerateInputSpec(), + func(conf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) { + input, err := newSqlSelectGenerateInput(conf, mgr, dbprovider, stopActivityChannel) + if err != nil { + return nil, err + } + return input, nil + }, + ) +} + +type sqlSelectGenerateInput struct { + driver string + dsn string + tableColsMap map[string][]string + provider DbPoolProvider + logger *service.Logger + + argsMapping *bloblang.Executor + + db mysql_queries.DBTX + dbMut sync.Mutex + // rows *sql.Rows + remaining int + index int + joinedRows []map[string]any + + shutSig *shutdown.Signaller + + stopActivityChannel chan error +} + +func newSqlSelectGenerateInput(conf *service.ParsedConfig, mgr *service.Resources, dbprovider DbPoolProvider, channel chan error) (*sqlSelectGenerateInput, error) { + driver, err := conf.FieldString("driver") + if err != nil { + return nil, err + } + dsn, err := conf.FieldString("dsn") + if err != nil { + return nil, err + } + + count, err := conf.FieldInt("count") + if err != nil { + return nil, err + } + + tmpMap, err := conf.FieldAnyMap("table_columns_map") + if err != nil { + return nil, err + } + tableColsMap := map[string][]string{} + for k, v := range tmpMap { + val, err := v.FieldStringList() + if err != nil { + return nil, err + } + tableColsMap[k] = val + } + var argsMapping *bloblang.Executor + if conf.Contains("args_mapping") { + argsMapping, err = conf.FieldBloblang("args_mapping") + if err != nil { + return nil, err + } + } + + return &sqlSelectGenerateInput{ + logger: mgr.Logger(), + shutSig: shutdown.NewSignaller(), + driver: driver, + dsn: dsn, + tableColsMap: tableColsMap, + argsMapping: argsMapping, + provider: dbprovider, + stopActivityChannel: channel, + remaining: count, + index: 0, + }, nil +} + +var _ service.Input = &pooledInput{} + +func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { + s.logger.Debug("connecting to pooled input") + s.dbMut.Lock() + defer s.dbMut.Unlock() + + if s.db != nil { + return nil + } + + seed, err := transformer_utils.GenerateCryptoSeed() + if err != nil { + return err + } + + db, err := s.provider.GetDb(s.driver, s.dsn) + if err != nil { + return nil + } + + var args []any + if s.argsMapping != nil { + iargs, err := s.argsMapping.Query(nil) + if err != nil { + return err + } + var ok bool + if args, ok = iargs.([]any); !ok { + return fmt.Errorf("mapping returned non-array result: %T", iargs) + } + } + + sqlRandomStr := "RANDOM()" + if s.driver == "mysql" { + sqlRandomStr = "RAND()" + } + + tables := []string{} + for t := range s.tableColsMap { + tables = append(tables, t) + } + + randomLimit, err := transformer_utils.GenerateRandomInt64InValueRange(10, 100) + if err != nil { + return err + } + + joinedRows := []map[string]any{} + + table := tables[0] + otherTables := tables[1:] + + cols := s.tableColsMap[table] + // update col names to be that of destination table or should it be handled on insert + selectColumns := make([]any, len(cols)) + for i, col := range cols { + selectColumns[i] = col + } + orderBy := exp.NewOrderedExpression(exp.NewLiteralExpression(sqlRandomStr), exp.AscDir, exp.NullsLastSortType) + builder := goqu.Dialect(s.driver) + selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomLimit)) + selectSql, _, err := selectBuilder.ToSQL() + if err != nil { + return err + } + + // select rows from + + rows, err := db.QueryContext(ctx, selectSql) + if err != nil { + if !neosync_benthos.IsMaxConnectionError(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return err + } + + rowObjList, err := sqlRowsToMapList(rows) + if err != nil { + _ = rows.Close() + return err + } + + for _, r := range rowObjList { + randLimit, err := transformer_utils.GenerateRandomInt64InValueRange(0, 3) + if err != nil { + return err + } + otherTableRows := [][]map[string]any{} + for _, t := range otherTables { + cols := s.tableColsMap[t] + selectColumns := make([]any, len(cols)) + for i, col := range cols { + selectColumns[i] = col + } + selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randLimit)) + selectSql, _, err := selectBuilder.ToSQL() + if err != nil { + return err + } + rows, err := db.QueryContext(ctx, selectSql) + if err != nil { + if !neosync_benthos.IsMaxConnectionError(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return err + } + rowObjList, err := sqlRowsToMapList(rows) + if err != nil { + _ = rows.Close() + return err + } + otherTableRows = append(otherTableRows, rowObjList) + } + + } + + s.joinedRows = joinedRows + go func() { + <-s.shutSig.CloseNowChan() + + s.dbMut.Lock() + if rows != nil { + _ = rows.Close() + rows = nil + } + // not closing the connection here as that is managed by an outside force + s.db = nil + s.dbMut.Unlock() + + s.shutSig.ShutdownComplete() + }() + return nil +} + +func (s *sqlSelectGenerateInput) Read(ctx context.Context) (*service.Message, service.AckFunc, error) { + s.dbMut.Lock() + defer s.dbMut.Unlock() + + if s.db == nil && s.rows == nil { + return nil, nil, service.ErrNotConnected + } + if s.rows == nil { + return nil, nil, service.ErrEndOfInput + } + if !s.rows.Next() { + err := s.rows.Err() + if err == nil { + err = service.ErrEndOfInput + } + _ = s.rows.Close() + s.rows = nil + return nil, nil, err + } + + obj, err := sqlRowToMap(s.rows) + if err != nil { + _ = s.rows.Close() + s.rows = nil + return nil, nil, err + } + + msg := service.NewMessage(nil) + msg.SetStructured(obj) + return msg, emptyAck, nil +} + +func (s *sqlSelectGenerateInput) Close(ctx context.Context) error { + s.shutSig.CloseNow() + s.dbMut.Lock() + isNil := s.db == nil + s.dbMut.Unlock() + if isNil { + return nil + } + select { + case <-s.shutSig.HasClosedChan(): + case <-ctx.Done(): + return ctx.Err() + } + return nil +} + +func sqlRowsToMapList(rows *sql.Rows) ([]map[string]any, error) { + results := []map[string]any{} + for rows.Next() { + obj, err := sqlRowToMap(rows) + if err != nil { + return nil, err + } + results = append(results, obj) + } + return results, nil +} + +func combineRows(maps []map[string]any) map[string]any { + result := make(map[string]any) + for _, m := range maps { + for k, v := range m { + result[k] = v + } + } + return result +} + +func combineRowLists(rows [][]map[string]any) []map[string]any { + results := []map[string]any{} + rowCount := len(rows[0]) + for i := 0; i < rowCount; i++ { + rowsToCombine := []map[string]any{} + for _, r := range rows { + rowsToCombine = append(rowsToCombine, r[i]) + } + results = append(results, combineRows(rowsToCombine)) + } + return results +} diff --git a/worker/pkg/workflows/datasync/workflow/workflow.go b/worker/pkg/workflows/datasync/workflow/workflow.go index 6472599b77..62e6ae7e5a 100644 --- a/worker/pkg/workflows/datasync/workflow/workflow.go +++ b/worker/pkg/workflows/datasync/workflow/workflow.go @@ -272,7 +272,7 @@ func invokeSync( settable.SetError(fmt.Errorf("unable to marshal benthos config: %w", err)) return } - + fmt.Println(string(configbits)) logger.Info("scheduling Sync for execution.") var result sync_activity.SyncResponse From 4956086e1dd992364cfb06d746de667b06606a85 Mon Sep 17 00:00:00 2001 From: Alisha Date: Fri, 12 Apr 2024 17:12:23 -0700 Subject: [PATCH 02/12] try batched input --- .../sql/input_generate_table_records.go | 71 +++------------ .../benthos/sql/input_sql_select_generate.go | 89 ++++++++++--------- 2 files changed, 56 insertions(+), 104 deletions(-) diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index dd4259455b..111f591524 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -4,8 +4,6 @@ package neosync_benthos_sql import ( "context" - "encoding/json" - "fmt" "sync" "github.com/benthosdev/benthos/v4/public/bloblang" @@ -13,7 +11,6 @@ import ( "github.com/benthosdev/benthos/v4/public/component/interop" "github.com/benthosdev/benthos/v4/public/service" mysql_queries "github.com/nucleuscloud/neosync/backend/gen/go/db/dbschemas/mysql" - neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" "github.com/nucleuscloud/neosync/worker/internal/benthos/shutdown" ) @@ -36,7 +33,7 @@ func RegisterGenerateTableRecordsInput(env *service.Environment, dbprovider DbPo // } // return input, nil nm := interop.UnwrapManagement(mgr) - b, err := newGenerateReaderFromParsed(conf, nm) + b, err := newGenerateReaderFromParsed(conf, nm, dbprovider, stopActivityChannel) if err != nil { return nil, err } @@ -72,7 +69,7 @@ type generateReader struct { stopActivityChannel chan error } -func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resources) (*generateReader, error) { +func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resources, dbprovider DbPoolProvider, channel chan error) (*generateReader, error) { driver, err := conf.FieldString("driver") if err != nil { return nil, err @@ -122,7 +119,7 @@ func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resour } // Connect establishes a Bloblang reader. -func (b *generateReader) Connect(ctx context.Context) error { +func (s *generateReader) Connect(ctx context.Context) error { s.logger.Debug("connecting to pooled input") s.dbMut.Lock() defer s.dbMut.Unlock() @@ -136,38 +133,12 @@ func (b *generateReader) Connect(ctx context.Context) error { return nil } - var args []any - if s.argsMapping != nil { - iargs, err := s.argsMapping.Query(nil) - if err != nil { - return err - } - var ok bool - if args, ok = iargs.([]any); !ok { - return fmt.Errorf("mapping returned non-array result: %T", iargs) - } - } + s.db = db - rows, err := db.QueryContext(ctx, s.queryStatic, args...) - if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { - s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) - s.stopActivityChannel <- err - } - return err - } - jsonF, _ := json.MarshalIndent(rows, "", " ") - fmt.Printf("\n rows: %s \n", string(jsonF)) - - s.rows = rows go func() { <-s.shutSig.CloseNowChan() s.dbMut.Lock() - if s.rows != nil { - _ = s.rows.Close() - s.rows = nil - } // not closing the connection here as that is managed by an outside force s.db = nil s.dbMut.Unlock() @@ -175,46 +146,24 @@ func (b *generateReader) Connect(ctx context.Context) error { s.shutSig.ShutdownComplete() }() return nil - return nil } // ReadBatch a new bloblang generated message. -func (b *generateReader) ReadBatch(ctx context.Context) (message.Batch, input.AsyncAckFn, error) { - batchSize := b.batchSize - if b.limited { - if b.remaining <= 0 { - return nil, nil, component.ErrTypeClosed - } - if b.remaining < batchSize { - batchSize = b.remaining - } +func (b *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, input.AsyncAckFn, error) { + if b.remaining <= 0 { + return nil, nil, service.ErrEndOfInput } - if !b.firstIsFree && b.timer != nil { - select { - case t, open := <-b.timer.C: - if !open { - return nil, nil, component.ErrTypeClosed - } - if b.schedule != nil { - b.timer.Reset(getDurationTillNextSchedule(t, *b.schedule, b.location)) - } - case <-ctx.Done(): - return nil, nil, component.ErrTimeout - } - } - b.firstIsFree = false + // b.firstIsFree = false - batch := make(message.Batch, 0, batchSize) + batch := make(service.MessageBatch, 0, batchSize) for i := 0; i < batchSize; i++ { p, err := b.exec.MapPart(0, batch) if err != nil { return nil, nil, err } if p != nil { - if b.limited { - b.remaining-- - } + b.remaining-- batch = append(batch, p) } } diff --git a/worker/internal/benthos/sql/input_sql_select_generate.go b/worker/internal/benthos/sql/input_sql_select_generate.go index b3d6a6347a..92b530527b 100644 --- a/worker/internal/benthos/sql/input_sql_select_generate.go +++ b/worker/internal/benthos/sql/input_sql_select_generate.go @@ -123,27 +123,22 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return nil } - seed, err := transformer_utils.GenerateCryptoSeed() - if err != nil { - return err - } - db, err := s.provider.GetDb(s.driver, s.dsn) if err != nil { return nil } - var args []any - if s.argsMapping != nil { - iargs, err := s.argsMapping.Query(nil) - if err != nil { - return err - } - var ok bool - if args, ok = iargs.([]any); !ok { - return fmt.Errorf("mapping returned non-array result: %T", iargs) - } - } + // var args []any + // if s.argsMapping != nil { + // iargs, err := s.argsMapping.Query(nil) + // if err != nil { + // return err + // } + // var ok bool + // if args, ok = iargs.([]any); !ok { + // return fmt.Errorf("mapping returned non-array result: %T", iargs) + // } + // } sqlRandomStr := "RANDOM()" if s.driver == "mysql" { @@ -179,8 +174,6 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return err } - // select rows from - rows, err := db.QueryContext(ctx, selectSql) if err != nil { if !neosync_benthos.IsMaxConnectionError(err.Error()) { @@ -228,7 +221,10 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { } otherTableRows = append(otherTableRows, rowObjList) } - + combinedRows := combineRowLists(otherTableRows) + for _, cr := range combinedRows { + joinedRows = append(joinedRows, combineRows([]map[string]any{r, cr})) + } } s.joinedRows = joinedRows @@ -253,32 +249,39 @@ func (s *sqlSelectGenerateInput) Read(ctx context.Context) (*service.Message, se s.dbMut.Lock() defer s.dbMut.Unlock() - if s.db == nil && s.rows == nil { - return nil, nil, service.ErrNotConnected - } - if s.rows == nil { + // if s.db == nil s.rows == nil { + // return nil, nil, service.ErrNotConnected + // } + // if s.rows == nil { + // return nil, nil, service.ErrEndOfInput + // } + // if !s.rows.Next() { + // err := s.rows.Err() + // if err == nil { + // err = service.ErrEndOfInput + // } + // _ = s.rows.Close() + // s.rows = nil + // return nil, nil, err + // } + // if s. + + // obj, err := sqlRowToMap(s.rows) + // if err != nil { + // _ = s.rows.Close() + // s.rows = nil + // return nil, nil, err + // } + + if s.index >= 0 && s.index < len(s.joinedRows) { + msg := service.NewMessage(nil) + msg.SetStructured(s.joinedRows[s.index]) + s.index++ + s.remaining-- + return msg, emptyAck, nil + } else { return nil, nil, service.ErrEndOfInput } - if !s.rows.Next() { - err := s.rows.Err() - if err == nil { - err = service.ErrEndOfInput - } - _ = s.rows.Close() - s.rows = nil - return nil, nil, err - } - - obj, err := sqlRowToMap(s.rows) - if err != nil { - _ = s.rows.Close() - s.rows = nil - return nil, nil, err - } - - msg := service.NewMessage(nil) - msg.SetStructured(obj) - return msg, emptyAck, nil } func (s *sqlSelectGenerateInput) Close(ctx context.Context) error { From 1b7dc7f0fa6c7c4f7b9da00df5d3327a8986da75 Mon Sep 17 00:00:00 2001 From: Alisha Date: Mon, 15 Apr 2024 12:19:58 -0700 Subject: [PATCH 03/12] child table benthos plugin --- .../sql/input_generate_table_records.go | 213 ++++++++++++++---- .../benthos/sql/input_sql_select_generate.go | 125 ++++------ 2 files changed, 212 insertions(+), 126 deletions(-) diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index 111f591524..3b725081f4 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -4,14 +4,19 @@ package neosync_benthos_sql import ( "context" + "database/sql" + "encoding/json" + "fmt" "sync" "github.com/benthosdev/benthos/v4/public/bloblang" - "github.com/benthosdev/benthos/v4/public/component/input" - "github.com/benthosdev/benthos/v4/public/component/interop" "github.com/benthosdev/benthos/v4/public/service" + "github.com/doug-martin/goqu/v9" + "github.com/doug-martin/goqu/v9/exp" mysql_queries "github.com/nucleuscloud/neosync/backend/gen/go/db/dbschemas/mysql" + neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" "github.com/nucleuscloud/neosync/worker/internal/benthos/shutdown" + transformer_utils "github.com/nucleuscloud/neosync/worker/internal/benthos/transformers/utils" ) func generateTableRecordsInputSpec() *service.ConfigSpec { @@ -21,27 +26,18 @@ func generateTableRecordsInputSpec() *service.ConfigSpec { Field(service.NewStringField("query")). Field(service.NewAnyMapField("table_columns_map")). Field(service.NewIntField("count")). - Field(service.NewBloblangField("args_mapping").Optional()) + Fields(service.NewBloblangField("mapping").Optional()) } + func RegisterGenerateTableRecordsInput(env *service.Environment, dbprovider DbPoolProvider, stopActivityChannel chan error) error { return env.RegisterBatchInput( "generate_table_records", generateTableRecordsInputSpec(), func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { - // input, err := newSqlSelectGenerateInput(conf, mgr, dbprovider, stopActivityChannel) - // if err != nil { - // return nil, err - // } - // return input, nil - nm := interop.UnwrapManagement(mgr) - b, err := newGenerateReaderFromParsed(conf, nm, dbprovider, stopActivityChannel) - if err != nil { - return nil, err - } - i, err := input.NewAsyncReader("generate", input.NewAsyncPreserver(b), nm) + b, err := newGenerateReaderFromParsed(conf, mgr, dbprovider, stopActivityChannel) if err != nil { return nil, err } - return interop.NewUnwrapInternalInput(i), nil + return service.AutoRetryNacksBatched(b), nil }, ) } @@ -55,14 +51,11 @@ type generateReader struct { provider DbPoolProvider logger *service.Logger - argsMapping *bloblang.Executor + mapping *bloblang.Executor - db mysql_queries.DBTX - dbMut sync.Mutex - // rows *sql.Rows - remaining int - index int - joinedRows []map[string]any + db mysql_queries.DBTX + dbMut sync.Mutex + remaining int shutSig *shutdown.Signaller @@ -96,9 +89,9 @@ func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resour } tableColsMap[k] = val } - var argsMapping *bloblang.Executor - if conf.Contains("args_mapping") { - argsMapping, err = conf.FieldBloblang("args_mapping") + var mapping *bloblang.Executor + if conf.Contains("mapping") { + mapping, err = conf.FieldBloblang("mapping") if err != nil { return nil, err } @@ -110,11 +103,10 @@ func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resour driver: driver, dsn: dsn, tableColsMap: tableColsMap, - argsMapping: argsMapping, + mapping: mapping, provider: dbprovider, stopActivityChannel: channel, remaining: count, - index: 0, }, nil } @@ -149,34 +141,167 @@ func (s *generateReader) Connect(ctx context.Context) error { } // ReadBatch a new bloblang generated message. -func (b *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, input.AsyncAckFn, error) { - if b.remaining <= 0 { +func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, input.AsyncAckFn, error) { + if s.remaining <= 0 { return nil, nil, service.ErrEndOfInput } - // b.firstIsFree = false + sqlRandomStr := "RANDOM()" + if s.driver == "mysql" { + sqlRandomStr = "RAND()" + } + + tables := []string{} + for t := range s.tableColsMap { + tables = append(tables, t) + } + + randomLrgLimit, err := transformer_utils.GenerateRandomInt64InValueRange(10, 50) + if err != nil { + return nil, nil, err + } + + table := tables[0] + otherTables := tables[1:] + + cols := s.tableColsMap[table] + // update col names to be that of destination table or should it be handled on insert + selectColumns := make([]any, len(cols)) + for i, col := range cols { + selectColumns[i] = col + } + orderBy := exp.NewOrderedExpression(exp.NewLiteralExpression(sqlRandomStr), exp.AscDir, exp.NullsLastSortType) + builder := goqu.Dialect(s.driver) + selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomLrgLimit)) + selectSql, _, err := selectBuilder.ToSQL() + if err != nil { + return nil, nil, err + } + + rows, err := s.db.QueryContext(ctx, selectSql) + if err != nil { + if !neosync_benthos.IsMaxConnectionError(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return nil, nil, err + } + + rowObjList, err := sqlRowsToMapList(rows) + if err != nil { + _ = rows.Close() + return nil, nil, err + } - batch := make(service.MessageBatch, 0, batchSize) - for i := 0; i < batchSize; i++ { - p, err := b.exec.MapPart(0, batch) + batch := service.MessageBatch{} + for _, r := range rowObjList { + randomSmLimit, err := transformer_utils.GenerateRandomInt64InValueRange(0, 3) if err != nil { return nil, nil, err } - if p != nil { - b.remaining-- - batch = append(batch, p) + otherTableRows := [][]map[string]any{} + for _, t := range otherTables { + cols := s.tableColsMap[t] + selectColumns := make([]any, len(cols)) + for i, col := range cols { + selectColumns[i] = col + } + selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomSmLimit)) + selectSql, _, err := selectBuilder.ToSQL() + if err != nil { + return nil, nil, err + } + rows, err := s.db.QueryContext(ctx, selectSql) + if err != nil { + if !neosync_benthos.IsMaxConnectionError(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return nil, nil, err + } + rowObjList, err := sqlRowsToMapList(rows) + if err != nil { + _ = rows.Close() + return nil, nil, err + } + otherTableRows = append(otherTableRows, rowObjList) + } + combinedRows := combineRowLists(otherTableRows) + for _, cr := range combinedRows { + var args map[string]any + if s.mapping != nil { + var iargs any + if iargs, err = s.mapping.Query(nil); err != nil { + return nil, nil, err + } + + var ok bool + if args, ok = iargs.(map[string]any); !ok { + err = fmt.Errorf("mapping returned non-array result: %T", iargs) + return nil, nil, err + } + } + newRow := combineRows([]map[string]any{r, cr, args}) + + rowStr, err := json.Marshal(newRow) + if err != nil { + return nil, nil, err + } + msg := service.NewMessage(rowStr) + batch = append(batch, msg) } } - if len(batch) == 0 { - return nil, nil, component.ErrTimeout - } + return batch, func(context.Context, error) error { return nil }, nil } -// CloseAsync shuts down the bloblang reader. -func (b *generateReader) Close(ctx context.Context) (err error) { - if b.timer != nil { - b.timer.Stop() +func (s *generateReader) Close(ctx context.Context) (err error) { + s.shutSig.CloseNow() + s.dbMut.Lock() + isNil := s.db == nil + s.dbMut.Unlock() + if isNil { + return nil + } + select { + case <-s.shutSig.HasClosedChan(): + case <-ctx.Done(): + return ctx.Err() + } + return nil +} + +func sqlRowsToMapList(rows *sql.Rows) ([]map[string]any, error) { + results := []map[string]any{} + for rows.Next() { + obj, err := sqlRowToMap(rows) + if err != nil { + return nil, err + } + results = append(results, obj) + } + return results, nil +} + +func combineRows(maps []map[string]any) map[string]any { + result := make(map[string]any) + for _, m := range maps { + for k, v := range m { + result[k] = v + } + } + return result +} + +func combineRowLists(rows [][]map[string]any) []map[string]any { + results := []map[string]any{} + rowCount := len(rows[0]) + for i := 0; i < rowCount; i++ { + rowsToCombine := []map[string]any{} + for _, r := range rows { + rowsToCombine = append(rowsToCombine, r[i]) + } + results = append(results, combineRows(rowsToCombine)) } - return + return results } diff --git a/worker/internal/benthos/sql/input_sql_select_generate.go b/worker/internal/benthos/sql/input_sql_select_generate.go index 92b530527b..eb037f8146 100644 --- a/worker/internal/benthos/sql/input_sql_select_generate.go +++ b/worker/internal/benthos/sql/input_sql_select_generate.go @@ -4,7 +4,6 @@ package neosync_benthos_sql import ( "context" - "database/sql" "fmt" "sync" @@ -123,22 +122,27 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return nil } + seed, err := transformer_utils.GenerateCryptoSeed() + if err != nil { + return err + } + db, err := s.provider.GetDb(s.driver, s.dsn) if err != nil { return nil } - // var args []any - // if s.argsMapping != nil { - // iargs, err := s.argsMapping.Query(nil) - // if err != nil { - // return err - // } - // var ok bool - // if args, ok = iargs.([]any); !ok { - // return fmt.Errorf("mapping returned non-array result: %T", iargs) - // } - // } + var args []any + if s.argsMapping != nil { + iargs, err := s.argsMapping.Query(nil) + if err != nil { + return err + } + var ok bool + if args, ok = iargs.([]any); !ok { + return fmt.Errorf("mapping returned non-array result: %T", iargs) + } + } sqlRandomStr := "RANDOM()" if s.driver == "mysql" { @@ -174,6 +178,8 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return err } + // select rows from + rows, err := db.QueryContext(ctx, selectSql) if err != nil { if !neosync_benthos.IsMaxConnectionError(err.Error()) { @@ -221,10 +227,7 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { } otherTableRows = append(otherTableRows, rowObjList) } - combinedRows := combineRowLists(otherTableRows) - for _, cr := range combinedRows { - joinedRows = append(joinedRows, combineRows([]map[string]any{r, cr})) - } + } s.joinedRows = joinedRows @@ -249,39 +252,32 @@ func (s *sqlSelectGenerateInput) Read(ctx context.Context) (*service.Message, se s.dbMut.Lock() defer s.dbMut.Unlock() - // if s.db == nil s.rows == nil { - // return nil, nil, service.ErrNotConnected - // } - // if s.rows == nil { - // return nil, nil, service.ErrEndOfInput - // } - // if !s.rows.Next() { - // err := s.rows.Err() - // if err == nil { - // err = service.ErrEndOfInput - // } - // _ = s.rows.Close() - // s.rows = nil - // return nil, nil, err - // } - // if s. - - // obj, err := sqlRowToMap(s.rows) - // if err != nil { - // _ = s.rows.Close() - // s.rows = nil - // return nil, nil, err - // } - - if s.index >= 0 && s.index < len(s.joinedRows) { - msg := service.NewMessage(nil) - msg.SetStructured(s.joinedRows[s.index]) - s.index++ - s.remaining-- - return msg, emptyAck, nil - } else { + if s.db == nil && s.rows == nil { + return nil, nil, service.ErrNotConnected + } + if s.rows == nil { return nil, nil, service.ErrEndOfInput } + if !s.rows.Next() { + err := s.rows.Err() + if err == nil { + err = service.ErrEndOfInput + } + _ = s.rows.Close() + s.rows = nil + return nil, nil, err + } + + obj, err := sqlRowToMap(s.rows) + if err != nil { + _ = s.rows.Close() + s.rows = nil + return nil, nil, err + } + + msg := service.NewMessage(nil) + msg.SetStructured(obj) + return msg, emptyAck, nil } func (s *sqlSelectGenerateInput) Close(ctx context.Context) error { @@ -299,38 +295,3 @@ func (s *sqlSelectGenerateInput) Close(ctx context.Context) error { } return nil } - -func sqlRowsToMapList(rows *sql.Rows) ([]map[string]any, error) { - results := []map[string]any{} - for rows.Next() { - obj, err := sqlRowToMap(rows) - if err != nil { - return nil, err - } - results = append(results, obj) - } - return results, nil -} - -func combineRows(maps []map[string]any) map[string]any { - result := make(map[string]any) - for _, m := range maps { - for k, v := range m { - result[k] = v - } - } - return result -} - -func combineRowLists(rows [][]map[string]any) []map[string]any { - results := []map[string]any{} - rowCount := len(rows[0]) - for i := 0; i < rowCount; i++ { - rowsToCombine := []map[string]any{} - for _, r := range rows { - rowsToCombine = append(rowsToCombine, r[i]) - } - results = append(results, combineRows(rowsToCombine)) - } - return results -} From afa8228e479471b0af61d15936a215d3eaf1e0b5 Mon Sep 17 00:00:00 2001 From: Alisha Date: Tue, 16 Apr 2024 11:34:13 -0700 Subject: [PATCH 04/12] stash --- .../sql/input_generate_table_records.go | 2 +- .../sql/input_generate_table_records_test.go | 51 +++++++++++ .../benthos/sql/input_sql_select_generate.go | 89 ++++++++++--------- .../gen-benthos-configs/benthos-builder.go | 30 +++++++ 4 files changed, 128 insertions(+), 44 deletions(-) create mode 100644 worker/internal/benthos/sql/input_generate_table_records_test.go diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index 3b725081f4..acaadc9f3f 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -141,7 +141,7 @@ func (s *generateReader) Connect(ctx context.Context) error { } // ReadBatch a new bloblang generated message. -func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, input.AsyncAckFn, error) { +func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { if s.remaining <= 0 { return nil, nil, service.ErrEndOfInput } diff --git a/worker/internal/benthos/sql/input_generate_table_records_test.go b/worker/internal/benthos/sql/input_generate_table_records_test.go new file mode 100644 index 0000000000..a5077dff65 --- /dev/null +++ b/worker/internal/benthos/sql/input_generate_table_records_test.go @@ -0,0 +1,51 @@ +package neosync_benthos_sql + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func Test_combineRows(t *testing.T) { + tests := []struct { + name string + maps []map[string]any + expected map[string]any + }{ + { + name: "empty input", + maps: []map[string]any{}, + expected: map[string]any{}, + }, + { + name: "single map", + maps: []map[string]any{ + {"key1": "value1", "key2": 2}, + }, + expected: map[string]any{"key1": "value1", "key2": 2}, + }, + { + name: "multiple maps with unique keys", + maps: []map[string]any{ + {"key1": "value1"}, + {"key2": "value2"}, + }, + expected: map[string]any{"key1": "value1", "key2": "value2"}, + }, + { + name: "multiple maps with overlapping keys", + maps: []map[string]any{ + {"key1": "value1", "key2": "value2"}, + {"key2": "newValue2", "key3": 3}, + }, + expected: map[string]any{"key1": "value1", "key2": "newValue2", "key3": 3}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + actual := combineRows(tt.maps) + assert.Equal(t, actual, tt.expected) + }) + } +} diff --git a/worker/internal/benthos/sql/input_sql_select_generate.go b/worker/internal/benthos/sql/input_sql_select_generate.go index eb037f8146..c9b7d6fba9 100644 --- a/worker/internal/benthos/sql/input_sql_select_generate.go +++ b/worker/internal/benthos/sql/input_sql_select_generate.go @@ -122,27 +122,22 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return nil } - seed, err := transformer_utils.GenerateCryptoSeed() - if err != nil { - return err - } - db, err := s.provider.GetDb(s.driver, s.dsn) if err != nil { return nil } - var args []any - if s.argsMapping != nil { - iargs, err := s.argsMapping.Query(nil) - if err != nil { - return err - } - var ok bool - if args, ok = iargs.([]any); !ok { - return fmt.Errorf("mapping returned non-array result: %T", iargs) - } - } + // var args []any + // if s.argsMapping != nil { + // iargs, err := s.argsMapping.Query(nil) + // if err != nil { + // return err + // } + // var ok bool + // if args, ok = iargs.([]any); !ok { + // return fmt.Errorf("mapping returned non-array result: %T", iargs) + // } + // } sqlRandomStr := "RANDOM()" if s.driver == "mysql" { @@ -178,8 +173,6 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { return err } - // select rows from - rows, err := db.QueryContext(ctx, selectSql) if err != nil { if !neosync_benthos.IsMaxConnectionError(err.Error()) { @@ -227,7 +220,10 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { } otherTableRows = append(otherTableRows, rowObjList) } - + combinedRows := combineRowLists(otherTableRows) + for _, cr := range combinedRows { + joinedRows = append(joinedRows, combineRows([]map[string]any{r, cr})) + } } s.joinedRows = joinedRows @@ -252,32 +248,39 @@ func (s *sqlSelectGenerateInput) Read(ctx context.Context) (*service.Message, se s.dbMut.Lock() defer s.dbMut.Unlock() - if s.db == nil && s.rows == nil { - return nil, nil, service.ErrNotConnected - } - if s.rows == nil { + // if s.db == nil s.rows == nil { + // return nil, nil, service.ErrNotConnected + // } + // if s.rows == nil { + // return nil, nil, service.ErrEndOfInput + // } + // if !s.rows.Next() { + // err := s.rows.Err() + // if err == nil { + // err = service.ErrEndOfInput + // } + // _ = s.rows.Close() + // s.rows = nil + // return nil, nil, err + // } + // if s. + + // obj, err := sqlRowToMap(s.rows) + // if err != nil { + // _ = s.rows.Close() + // s.rows = nil + // return nil, nil, err + // } + + if s.index >= 0 && s.index < len(s.joinedRows) { + msg := service.NewMessage(nil) + msg.SetStructured(s.joinedRows[s.index]) + s.index++ + s.remaining-- + return msg, emptyAck, nil + } else { return nil, nil, service.ErrEndOfInput } - if !s.rows.Next() { - err := s.rows.Err() - if err == nil { - err = service.ErrEndOfInput - } - _ = s.rows.Close() - s.rows = nil - return nil, nil, err - } - - obj, err := sqlRowToMap(s.rows) - if err != nil { - _ = s.rows.Close() - s.rows = nil - return nil, nil, err - } - - msg := service.NewMessage(nil) - msg.SetStructured(obj) - return msg, emptyAck, nil } func (s *sqlSelectGenerateInput) Close(ctx context.Context) error { diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index 5e19c93070..ebc8da4d47 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -105,6 +105,36 @@ func (b *benthosBuilder) GenerateBenthosConfigs( case *mgmtv1alpha1.JobSourceOptions_Generate: sourceTableOpts := groupGenerateSourceOptionsByTable(jobSourceConfig.Generate.Schemas) // TODO this needs to be updated to get db schema + // get depenendcy configs + // split root tables vs children tables + // ////////////////////////////////////////////////////////////////////// + + sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) + if err != nil { + return nil, fmt.Errorf("unable to get connection by id: %w", err) + } + db, err := b.sqladapter.NewSqlDb(ctx, slogger, sourceConnection) + if err != nil { + return nil, fmt.Errorf("unable to create new sql db: %w", err) + } + defer db.ClosePool() + allConstraints, err := db.GetAllForeignKeyConstraints(ctx, uniqueSchemas) + if err != nil { + return nil, fmt.Errorf("unable to retrieve database foreign key constraints: %w", err) + } + slogger.Info(fmt.Sprintf("found %d foreign key constraints for database", len(allConstraints))) + td := sql_manager.GetDbTableDependencies(allConstraints) + + primaryKeys, err := db.GetAllPrimaryKeyConstraints(ctx, uniqueSchemas) + if err != nil { + return nil, fmt.Errorf("unable to get all primary key constraints: %w", err) + } + primaryKeyMap := sql_manager.GetTablePrimaryKeysMap(primaryKeys) + + tables := filterNullTables(groupedMappings) + dependencyConfigs := tabledependency.GetRunConfigs(td, tables, map[string]string{}) + + // ///////////////////////////////////////////////////////////// sourceResponses, err := buildBenthosGenerateSourceConfigResponses(ctx, b.transformerclient, groupedMappings, sourceTableOpts, map[string]*dbschemas_utils.ColumnInfo{}) if err != nil { return nil, fmt.Errorf("unable to build benthos generate source config responses: %w", err) From 1ccf60fe995e860e0e71dbfdb5d8b1d829ff0bb5 Mon Sep 17 00:00:00 2001 From: Alisha Date: Wed, 17 Apr 2024 12:55:46 -0700 Subject: [PATCH 05/12] stash --- worker/internal/benthos/config.go | 16 +- .../sql/input_generate_table_records.go | 37 ++- .../gen-benthos-configs/benthos-builder.go | 272 +++--------------- .../gen-benthos-configs/generate-builder.go | 176 ++++++++++++ .../gen-benthos-configs/processors.go | 138 +++++++++ 5 files changed, 400 insertions(+), 239 deletions(-) create mode 100644 worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go create mode 100644 worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go diff --git a/worker/internal/benthos/config.go b/worker/internal/benthos/config.go index a30fd9be5e..e645f5efa2 100644 --- a/worker/internal/benthos/config.go +++ b/worker/internal/benthos/config.go @@ -54,9 +54,19 @@ type InputConfig struct { } type Inputs struct { - SqlSelect *SqlSelect `json:"sql_select,omitempty" yaml:"sql_select,omitempty"` - PooledSqlRaw *InputPooledSqlRaw `json:"pooled_sql_raw,omitempty" yaml:"pooled_sql_raw,omitempty"` - Generate *Generate `json:"generate,omitempty" yaml:"generate,omitempty"` + SqlSelect *SqlSelect `json:"sql_select,omitempty" yaml:"sql_select,omitempty"` + PooledSqlRaw *InputPooledSqlRaw `json:"pooled_sql_raw,omitempty" yaml:"pooled_sql_raw,omitempty"` + Generate *Generate `json:"generate,omitempty" yaml:"generate,omitempty"` + GenerateSqlSelect *GenerateSqlSelect `json:"generate_sql_select,omitempty" yaml:"generate_sql_select,omitempty"` +} + +type GenerateSqlSelect struct { + Mapping string `json:"mapping" yaml:"mapping"` + Count int `json:"count" yaml:"count"` + Driver string `json:"driver" yaml:"driver"` + Dsn string `json:"dsn" yaml:"dsn"` + TableColumnsMap map[string][]string `json:"table_columns_map" yaml:"table_columns_map"` + ColumnNameMap map[string]string `json:"column_name_map,omitempty" yaml:"column_name_map,omitempty"` } type Generate struct { diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index acaadc9f3f..02d61a2086 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -23,8 +23,8 @@ func generateTableRecordsInputSpec() *service.ConfigSpec { return service.NewConfigSpec(). Field(service.NewStringField("driver")). Field(service.NewStringField("dsn")). - Field(service.NewStringField("query")). Field(service.NewAnyMapField("table_columns_map")). + Field(service.NewStringMapField("column_name_map").Optional().Example("{ schema.table.column: new_column_name }")). Field(service.NewIntField("count")). Fields(service.NewBloblangField("mapping").Optional()) } @@ -45,11 +45,12 @@ func RegisterGenerateTableRecordsInput(env *service.Environment, dbprovider DbPo //------------------------------------------------------------------------------ type generateReader struct { - driver string - dsn string - tableColsMap map[string][]string - provider DbPoolProvider - logger *service.Logger + driver string + dsn string + tableColsMap map[string][]string + columnNameMap map[string]string + provider DbPoolProvider + logger *service.Logger mapping *bloblang.Executor @@ -89,6 +90,15 @@ func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resour } tableColsMap[k] = val } + + columnNameMap := map[string]string{} + if conf.Contains("column_name_map") { + columnNameMap, err = conf.FieldStringMap("column_name_map") + if err != nil { + return nil, err + } + } + var mapping *bloblang.Executor if conf.Contains("mapping") { mapping, err = conf.FieldBloblang("mapping") @@ -103,6 +113,7 @@ func newGenerateReaderFromParsed(conf *service.ParsedConfig, mgr *service.Resour driver: driver, dsn: dsn, tableColsMap: tableColsMap, + columnNameMap: columnNameMap, mapping: mapping, provider: dbprovider, stopActivityChannel: channel, @@ -168,7 +179,12 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s // update col names to be that of destination table or should it be handled on insert selectColumns := make([]any, len(cols)) for i, col := range cols { - selectColumns[i] = col + as, ok := s.columnNameMap[fmt.Sprintf("%s.%s", table, col)] + if ok { + selectColumns[i] = goqu.I(col).As(as) + } else { + selectColumns[i] = col + } } orderBy := exp.NewOrderedExpression(exp.NewLiteralExpression(sqlRandomStr), exp.AscDir, exp.NullsLastSortType) builder := goqu.Dialect(s.driver) @@ -204,7 +220,12 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s cols := s.tableColsMap[t] selectColumns := make([]any, len(cols)) for i, col := range cols { - selectColumns[i] = col + as, ok := s.columnNameMap[fmt.Sprintf("%s.%s", t, col)] + if ok { + selectColumns[i] = goqu.I(col).As(as) + } else { + selectColumns[i] = col + } } selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomSmLimit)) selectSql, _, err := selectBuilder.ToSQL() diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index ebc8da4d47..e66e61486a 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -107,7 +107,7 @@ func (b *benthosBuilder) GenerateBenthosConfigs( // TODO this needs to be updated to get db schema // get depenendcy configs // split root tables vs children tables - // ////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////// sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) if err != nil { @@ -133,14 +133,55 @@ func (b *benthosBuilder) GenerateBenthosConfigs( tables := filterNullTables(groupedMappings) dependencyConfigs := tabledependency.GetRunConfigs(td, tables, map[string]string{}) + dependencyMap := map[string][]*tabledependency.RunConfig{} + for _, cfg := range dependencyConfigs { + _, ok := dependencyMap[cfg.Table] + if ok { + dependencyMap[cfg.Table] = append(dependencyMap[cfg.Table], cfg) + } else { + dependencyMap[cfg.Table] = []*tabledependency.RunConfig{cfg} + } + } - // ///////////////////////////////////////////////////////////// - sourceResponses, err := buildBenthosGenerateSourceConfigResponses(ctx, b.transformerclient, groupedMappings, sourceTableOpts, map[string]*dbschemas_utils.ColumnInfo{}) + /////////////////////////////////////////////////////////////// + + // root tables + sourceResponses, err := buildBenthosGenerateSourceConfigResponses(ctx, b.transformerclient, groupedMappings, sourceTableOpts, map[string]*dbschemas_utils.ColumnInfo{}, dependencyMap, db.Driver, sourceConnection.Id, primaryKeyMap) if err != nil { return nil, fmt.Errorf("unable to build benthos generate source config responses: %w", err) } responses = append(responses, sourceResponses...) + /////////////////////////////////////////////////////////////// + + // for _, resp := range responses { + // tableName := neosync_benthos.BuildBenthosTable(resp.TableSchema, resp.TableName) + // configs := dependencyMap[tableName] + // if len(configs) > 1 { + // // circular dependency + // for _, c := range configs { + // if c.Columns != nil && c.Columns.Exclude != nil && len(c.Columns.Exclude) > 0 { + // resp.excludeColumns = c.Columns.Exclude + // resp.DependsOn = c.DependsOn + // } else if c.Columns != nil && c.Columns.Include != nil && len(c.Columns.Include) > 0 { + // pks := primaryKeyMap[tableName] + // if len(pks) == 0 { + // return nil, fmt.Errorf("no primary keys found for table (%s). Unable to build update query", tableName) + // } + + // // config for sql update + // resp.updateConfig = c + // resp.primaryKeys = pks + // } + // } + // } else if len(configs) == 1 { + // resp.DependsOn = configs[0].DependsOn + // } else { + // return nil, fmt.Errorf("unexpected number of dependency configs") + // } + // } + /////////////////////////////////////////////////////////////// + case *mgmtv1alpha1.JobSourceOptions_Postgres, *mgmtv1alpha1.JobSourceOptions_Mysql: sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) if err != nil { @@ -550,107 +591,6 @@ func buildOutputArgs(resp *BenthosConfigResponse, tm *tableMapping) *sqlUpdateOu } } -type generateSourceTableOptions struct { - Count int -} - -func buildBenthosGenerateSourceConfigResponses( - ctx context.Context, - transformerclient mgmtv1alpha1connect.TransformersServiceClient, - mappings []*tableMapping, - sourceTableOpts map[string]*generateSourceTableOptions, - columnInfo map[string]*dbschemas_utils.ColumnInfo, -) ([]*BenthosConfigResponse, error) { - responses := []*BenthosConfigResponse{} - - for _, tableMapping := range mappings { - if shared.AreAllColsNull(tableMapping.Mappings) { - // skiping table as no columns are mapped - continue - } - - var count = 0 - tableOpt := sourceTableOpts[neosync_benthos.BuildBenthosTable(tableMapping.Schema, tableMapping.Table)] - if tableOpt != nil { - count = tableOpt.Count - } - - jsCode, err := extractJsFunctionsAndOutputs(ctx, transformerclient, tableMapping.Mappings) - if err != nil { - return nil, err - } - - mutations, err := buildMutationConfigs(ctx, transformerclient, tableMapping.Mappings, columnInfo) - if err != nil { - return nil, err - } - var processors []*neosync_benthos.ProcessorConfig - // for the generate input, benthos requires a mapping, so falling back to a - // generic empty object if the mutations are empty - if mutations == "" { - mutations = "root = {}" - } - processors = append(processors, &neosync_benthos.ProcessorConfig{Mutation: &mutations}) - - if jsCode != "" { - processors = append(processors, &neosync_benthos.ProcessorConfig{Javascript: &neosync_benthos.JavascriptConfig{Code: jsCode}}) - } - if len(processors) > 0 { - // add catch and error processor - processors = append(processors, &neosync_benthos.ProcessorConfig{Catch: []*neosync_benthos.ProcessorConfig{ - {Error: &neosync_benthos.ErrorProcessorConfig{ - ErrorMsg: `${! meta("fallback_error")}`, - }}, - }}) - } - - bc := &neosync_benthos.BenthosConfig{ - StreamConfig: neosync_benthos.StreamConfig{ - Input: &neosync_benthos.InputConfig{ - Inputs: neosync_benthos.Inputs{ - Generate: &neosync_benthos.Generate{ - Interval: "", - Count: count, - Mapping: "root = {}", - }, - }, - }, - Pipeline: &neosync_benthos.PipelineConfig{ - Threads: -1, - Processors: []neosync_benthos.ProcessorConfig{}, // leave empty. processors should be on output - }, - Output: &neosync_benthos.OutputConfig{ - Outputs: neosync_benthos.Outputs{ - Broker: &neosync_benthos.OutputBrokerConfig{ - Pattern: "fan_out", - Outputs: []neosync_benthos.Outputs{}, - }, - }, - }, - }, - } - - responses = append(responses, &BenthosConfigResponse{ - Name: neosync_benthos.BuildBenthosTable(tableMapping.Schema, tableMapping.Table), // todo: may need to expand on this - Config: bc, - DependsOn: []*tabledependency.DependsOn{}, - - TableSchema: tableMapping.Schema, - TableName: tableMapping.Table, - - Processors: processors, - - metriclabels: metrics.MetricLabels{ - metrics.NewEqLabel(metrics.TableSchemaLabel, tableMapping.Schema), - metrics.NewEqLabel(metrics.TableNameLabel, tableMapping.Table), - metrics.NewEqLabel(metrics.JobTypeLabel, "generate"), - }, - }) - } - - return responses, nil -} - func (b *benthosBuilder) getJobById( ctx context.Context, jobId string, @@ -1259,130 +1199,6 @@ type tableMapping struct { Mappings []*mgmtv1alpha1.JobMapping } -func buildProcessorConfigs( - ctx context.Context, - transformerclient mgmtv1alpha1connect.TransformersServiceClient, - cols []*mgmtv1alpha1.JobMapping, - tableColumnInfo map[string]*dbschemas_utils.ColumnInfo, - columnConstraints map[string]*dbschemas_utils.ForeignKey, - primaryKeys []string, - jobId, runId string, - redisConfig *shared.RedisConfig, -) ([]*neosync_benthos.ProcessorConfig, error) { - jsCode, err := extractJsFunctionsAndOutputs(ctx, transformerclient, cols) - if err != nil { - return nil, err - } - - mutations, err := buildMutationConfigs(ctx, transformerclient, cols, tableColumnInfo) - if err != nil { - return nil, err - } - - cacheBranches, err := buildBranchCacheConfigs(cols, columnConstraints, jobId, runId, redisConfig) - if err != nil { - return nil, err - } - - pkMapping := buildPrimaryKeyMappingConfigs(cols, primaryKeys) - - var processorConfigs []*neosync_benthos.ProcessorConfig - if pkMapping != "" { - processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Mapping: &pkMapping}) - } - if mutations != "" { - processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Mutation: &mutations}) - } - if jsCode != "" { - processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Javascript: &neosync_benthos.JavascriptConfig{Code: jsCode}}) - } - if len(cacheBranches) > 0 { - for _, config := range cacheBranches { - processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Branch: config}) - } - } - - if len(processorConfigs) > 0 { - // add catch and error processor - processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Catch: []*neosync_benthos.ProcessorConfig{ - {Error: &neosync_benthos.ErrorProcessorConfig{ - ErrorMsg: `${! meta("fallback_error")}`, - }}, - }}) - } - - return processorConfigs, err -} - -func extractJsFunctionsAndOutputs(ctx context.Context, transformerclient mgmtv1alpha1connect.TransformersServiceClient, cols []*mgmtv1alpha1.JobMapping) (string, error) { - var benthosOutputs []string - var jsFunctions []string - - for _, col := range cols { - if shouldProcessStrict(col.Transformer) { - if _, ok := col.Transformer.Config.Config.(*mgmtv1alpha1.TransformerConfig_UserDefinedTransformerConfig); ok { - val, err := convertUserDefinedFunctionConfig(ctx, transformerclient, col.Transformer) - if err != nil { - return "", errors.New("unable to look up user defined transformer config by id") - } - col.Transformer = val - } - switch col.Transformer.Source { - case mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT: - code := col.Transformer.Config.GetTransformJavascriptConfig().Code - if code != "" { - jsFunctions = append(jsFunctions, constructJsFunction(code, col.Column, col.Transformer.Source)) - benthosOutputs = append(benthosOutputs, constructBenthosJavascriptObject(col.Column, col.Transformer.Source)) - } - case mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT: - code := col.Transformer.Config.GetGenerateJavascriptConfig().Code - if code != "" { - jsFunctions = append(jsFunctions, constructJsFunction(code, col.Column, col.Transformer.Source)) - benthosOutputs = append(benthosOutputs, constructBenthosJavascriptObject(col.Column, col.Transformer.Source)) - } - } - } - } - - if len(jsFunctions) > 0 { - return constructBenthosJsProcessor(jsFunctions, benthosOutputs), nil - } else { - return "", nil - } -} - -func buildMutationConfigs( - ctx context.Context, - transformerclient mgmtv1alpha1connect.TransformersServiceClient, - cols []*mgmtv1alpha1.JobMapping, - tableColumnInfo map[string]*dbschemas_utils.ColumnInfo, -) (string, error) { - mutations := []string{} - - for _, col := range cols { - colInfo := tableColumnInfo[col.Column] - if shouldProcessColumn(col.Transformer) { - if _, ok := col.Transformer.Config.Config.(*mgmtv1alpha1.TransformerConfig_UserDefinedTransformerConfig); ok { - // handle user defined transformer -> get the user defined transformer configs using the id - val, err := convertUserDefinedFunctionConfig(ctx, transformerclient, col.Transformer) - if err != nil { - return "", errors.New("unable to look up user defined transformer config by id") - } - col.Transformer = val - } - if col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT && col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT { - mutation, err := computeMutationFunction(col, colInfo) - if err != nil { - return "", fmt.Errorf("%s is not a supported transformer: %w", col.Transformer, err) - } - mutations = append(mutations, fmt.Sprintf("root.%q = %s", col.Column, mutation)) - } - } - } - - return strings.Join(mutations, "\n"), nil -} - func buildPrimaryKeyMappingConfigs(cols []*mgmtv1alpha1.JobMapping, primaryKeys []string) string { mappings := []string{} for _, col := range cols { diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go new file mode 100644 index 0000000000..67a9b315ac --- /dev/null +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go @@ -0,0 +1,176 @@ +package genbenthosconfigs_activity + +import ( + "context" + "fmt" + + "github.com/nucleuscloud/neosync/backend/gen/go/protos/mgmt/v1alpha1/mgmtv1alpha1connect" + dbschemas_utils "github.com/nucleuscloud/neosync/backend/pkg/dbschemas" + "github.com/nucleuscloud/neosync/backend/pkg/metrics" + tabledependency "github.com/nucleuscloud/neosync/backend/pkg/table-dependency" + neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" + "github.com/nucleuscloud/neosync/worker/pkg/workflows/datasync/activities/shared" +) + +type generateSourceTableOptions struct { + Count int +} + +func buildBenthosGenerateSourceConfigResponses( + ctx context.Context, + transformerclient mgmtv1alpha1connect.TransformersServiceClient, + mappings []*tableMapping, + sourceTableOpts map[string]*generateSourceTableOptions, + columnInfo map[string]*dbschemas_utils.ColumnInfo, + dependencyMap map[string][]*tabledependency.RunConfig, + driver, dsnConnectionId string, + primaryKeyMap map[string][]string, +) ([]*BenthosConfigResponse, error) { + responses := []*BenthosConfigResponse{} + + // root tables + for _, tableMapping := range mappings { + if shared.AreAllColsNull(tableMapping.Mappings) { + // skiping table as no columns are mapped + continue + } + + tableName := neosync_benthos.BuildBenthosTable(tableMapping.Schema, tableMapping.Table) + runConfigs := dependencyMap[tableName] + + var count = 0 + tableOpt := sourceTableOpts[tableName] + if tableOpt != nil { + count = tableOpt.Count + } + + jsCode, err := extractJsFunctionsAndOutputs(ctx, transformerclient, tableMapping.Mappings) + if err != nil { + return nil, err + } + + mutations, err := buildMutationConfigs(ctx, transformerclient, tableMapping.Mappings, columnInfo) + if err != nil { + return nil, err + } + var processors []neosync_benthos.ProcessorConfig + // for the generate input, benthos requires a mapping, so falling back to a + // generic empty object if the mutations are empty + if mutations == "" { + mutations = "root = {}" + } + processors = append(processors, neosync_benthos.ProcessorConfig{Mutation: &mutations}) + + if jsCode != "" { + processors = append(processors, neosync_benthos.ProcessorConfig{Javascript: &neosync_benthos.JavascriptConfig{Code: jsCode}}) + } + if len(processors) > 0 { + // add catch and error processor + processors = append(processors, neosync_benthos.ProcessorConfig{Catch: []*neosync_benthos.ProcessorConfig{ + {Error: &neosync_benthos.ErrorProcessorConfig{ + ErrorMsg: `${! meta("fallback_error")}`, + }}, + }}) + } + + var bc *neosync_benthos.BenthosConfig + if len(runConfigs) > 0 && len(runConfigs[0].DependsOn) > 0 { + bc = &neosync_benthos.BenthosConfig{ + StreamConfig: neosync_benthos.StreamConfig{ + Input: &neosync_benthos.InputConfig{ + Inputs: neosync_benthos.Inputs{ + GenerateSqlSelect: &neosync_benthos.GenerateSqlSelect{ + Count: count, + Mapping: mutations, + Driver: driver, + Dsn: "${SOURCE_CONNECTION_DSN}", + // TableColumnsMap: , + // ColumnNameMap:, + }, + }, + }, + Pipeline: &neosync_benthos.PipelineConfig{ + Threads: -1, + Processors: processors, + }, + Output: &neosync_benthos.OutputConfig{ + Outputs: neosync_benthos.Outputs{ + Broker: &neosync_benthos.OutputBrokerConfig{ + Pattern: "fan_out_sequential_fail_fast", + Outputs: []neosync_benthos.Outputs{}, + }, + }, + }, + }, + } + } else { + bc = &neosync_benthos.BenthosConfig{ + StreamConfig: neosync_benthos.StreamConfig{ + Input: &neosync_benthos.InputConfig{ + Inputs: neosync_benthos.Inputs{ + Generate: &neosync_benthos.Generate{ + Interval: "", + Count: count, + Mapping: "root = {}", + }, + }, + }, + Pipeline: &neosync_benthos.PipelineConfig{ + Threads: -1, + Processors: processors, + }, + Output: &neosync_benthos.OutputConfig{ + Outputs: neosync_benthos.Outputs{ + Broker: &neosync_benthos.OutputBrokerConfig{ + Pattern: "fan_out_sequential_fail_fast", + Outputs: []neosync_benthos.Outputs{}, + }, + }, + }, + }, + } + } + + resp := &BenthosConfigResponse{ + Name: neosync_benthos.BuildBenthosTable(tableMapping.Schema, tableMapping.Table), // todo: may need to expand on this + Config: bc, + DependsOn: []*tabledependency.DependsOn{}, + BenthosDsns: []*shared.BenthosDsn{{ConnectionId: dsnConnectionId, EnvVarKey: "SOURCE_CONNECTION_DSN"}}, + + TableSchema: tableMapping.Schema, + TableName: tableMapping.Table, + + // Processors: processors, + + metriclabels: metrics.MetricLabels{ + metrics.NewEqLabel(metrics.TableSchemaLabel, tableMapping.Schema), + metrics.NewEqLabel(metrics.TableNameLabel, tableMapping.Table), + metrics.NewEqLabel(metrics.JobTypeLabel, "generate"), + }, + } + if len(runConfigs) > 1 { + // circular dependency + for _, c := range runConfigs { + if c.Columns != nil && c.Columns.Exclude != nil && len(c.Columns.Exclude) > 0 { + resp.excludeColumns = c.Columns.Exclude + resp.DependsOn = c.DependsOn + } else if c.Columns != nil && c.Columns.Include != nil && len(c.Columns.Include) > 0 { + pks := primaryKeyMap[tableName] + if len(pks) == 0 { + return nil, fmt.Errorf("no primary keys found for table (%s). Unable to build update query", tableName) + } + + // config for sql update + resp.updateConfig = c + resp.primaryKeys = pks + } + } + } else if len(runConfigs) == 1 { + resp.DependsOn = runConfigs[0].DependsOn + } + + responses = append(responses, resp) + } + + return responses, nil +} diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go new file mode 100644 index 0000000000..551e3a96a4 --- /dev/null +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go @@ -0,0 +1,138 @@ +package genbenthosconfigs_activity + +import ( + "context" + "errors" + "fmt" + "strings" + + mgmtv1alpha1 "github.com/nucleuscloud/neosync/backend/gen/go/protos/mgmt/v1alpha1" + "github.com/nucleuscloud/neosync/backend/gen/go/protos/mgmt/v1alpha1/mgmtv1alpha1connect" + dbschemas_utils "github.com/nucleuscloud/neosync/backend/pkg/dbschemas" + neosync_benthos "github.com/nucleuscloud/neosync/worker/internal/benthos" + "github.com/nucleuscloud/neosync/worker/pkg/workflows/datasync/activities/shared" +) + +func buildProcessorConfigs( + ctx context.Context, + transformerclient mgmtv1alpha1connect.TransformersServiceClient, + cols []*mgmtv1alpha1.JobMapping, + tableColumnInfo map[string]*dbschemas_utils.ColumnInfo, + columnConstraints map[string]*dbschemas_utils.ForeignKey, + primaryKeys []string, + jobId, runId string, + redisConfig *shared.RedisConfig, +) ([]*neosync_benthos.ProcessorConfig, error) { + jsCode, err := extractJsFunctionsAndOutputs(ctx, transformerclient, cols) + if err != nil { + return nil, err + } + + mutations, err := buildMutationConfigs(ctx, transformerclient, cols, tableColumnInfo) + if err != nil { + return nil, err + } + + cacheBranches, err := buildBranchCacheConfigs(cols, columnConstraints, jobId, runId, redisConfig) + if err != nil { + return nil, err + } + + pkMapping := buildPrimaryKeyMappingConfigs(cols, primaryKeys) + + var processorConfigs []*neosync_benthos.ProcessorConfig + if pkMapping != "" { + processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Mapping: &pkMapping}) + } + if mutations != "" { + processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Mutation: &mutations}) + } + if jsCode != "" { + processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Javascript: &neosync_benthos.JavascriptConfig{Code: jsCode}}) + } + if len(cacheBranches) > 0 { + for _, config := range cacheBranches { + processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Branch: config}) + } + } + + if len(processorConfigs) > 0 { + // add catch and error processor + processorConfigs = append(processorConfigs, &neosync_benthos.ProcessorConfig{Catch: []*neosync_benthos.ProcessorConfig{ + {Error: &neosync_benthos.ErrorProcessorConfig{ + ErrorMsg: `${! meta("fallback_error")}`, + }}, + }}) + } + + return processorConfigs, err +} + +func extractJsFunctionsAndOutputs(ctx context.Context, transformerclient mgmtv1alpha1connect.TransformersServiceClient, cols []*mgmtv1alpha1.JobMapping) (string, error) { + var benthosOutputs []string + var jsFunctions []string + + for _, col := range cols { + if shouldProcessStrict(col.Transformer) { + if _, ok := col.Transformer.Config.Config.(*mgmtv1alpha1.TransformerConfig_UserDefinedTransformerConfig); ok { + val, err := convertUserDefinedFunctionConfig(ctx, transformerclient, col.Transformer) + if err != nil { + return "", errors.New("unable to look up user defined transformer config by id") + } + col.Transformer = val + } + switch col.Transformer.Source { + case mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT: + code := col.Transformer.Config.GetTransformJavascriptConfig().Code + if code != "" { + jsFunctions = append(jsFunctions, constructJsFunction(code, col.Column, col.Transformer.Source)) + benthosOutputs = append(benthosOutputs, constructBenthosJavascriptObject(col.Column, col.Transformer.Source)) + } + case mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT: + code := col.Transformer.Config.GetGenerateJavascriptConfig().Code + if code != "" { + jsFunctions = append(jsFunctions, constructJsFunction(code, col.Column, col.Transformer.Source)) + benthosOutputs = append(benthosOutputs, constructBenthosJavascriptObject(col.Column, col.Transformer.Source)) + } + } + } + } + + if len(jsFunctions) > 0 { + return constructBenthosJsProcessor(jsFunctions, benthosOutputs), nil + } else { + return "", nil + } +} + +func buildMutationConfigs( + ctx context.Context, + transformerclient mgmtv1alpha1connect.TransformersServiceClient, + cols []*mgmtv1alpha1.JobMapping, + tableColumnInfo map[string]*dbschemas_utils.ColumnInfo, +) (string, error) { + mutations := []string{} + + for _, col := range cols { + colInfo := tableColumnInfo[col.Column] + if shouldProcessColumn(col.Transformer) { + if _, ok := col.Transformer.Config.Config.(*mgmtv1alpha1.TransformerConfig_UserDefinedTransformerConfig); ok { + // handle user defined transformer -> get the user defined transformer configs using the id + val, err := convertUserDefinedFunctionConfig(ctx, transformerclient, col.Transformer) + if err != nil { + return "", errors.New("unable to look up user defined transformer config by id") + } + col.Transformer = val + } + if col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT && col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT { + mutation, err := computeMutationFunction(col, colInfo) + if err != nil { + return "", fmt.Errorf("%s is not a supported transformer: %w", col.Transformer, err) + } + mutations = append(mutations, fmt.Sprintf("root.%q = %s", col.Column, mutation)) + } + } + } + + return strings.Join(mutations, "\n"), nil +} From 11feb3a6d8080fd9a7e84aeaf5731e0eb68ce39c Mon Sep 17 00:00:00 2001 From: Alisha Date: Thu, 18 Apr 2024 10:29:13 -0700 Subject: [PATCH 06/12] allow multi tables on frontend --- .../new/job/generate/single/schema/page.tsx | 14 +-- .../app/(mgmt)/[account]/new/job/schema.ts | 8 ++ .../jobs/SchemaTable/SchemaTable.tsx | 2 +- .../gen-benthos-configs/benthos-builder.go | 93 +++++++------------ .../gen-benthos-configs/generate-builder.go | 23 +++-- 5 files changed, 64 insertions(+), 76 deletions(-) diff --git a/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx b/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx index b2fb792b54..1919176bdc 100644 --- a/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx +++ b/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx @@ -57,9 +57,9 @@ import { useSessionStorage } from 'usehooks-ts'; import JobsProgressSteps, { DATA_GEN_STEPS } from '../../../JobsProgressSteps'; import { DefineFormValues, - SINGLE_TABLE_SCHEMA_FORM_SCHEMA, + MULTI_TABLE_SCHEMA_FORM_SCHEMA, + MultiTableSchemaFormValues, SingleTableConnectFormValues, - SingleTableSchemaFormValues, } from '../../../schema'; const isBrowser = () => typeof window !== 'undefined'; @@ -98,7 +98,7 @@ export default function Page({ searchParams }: PageProps): ReactElement { const formKey = `${sessionPrefix}-new-job-single-table-schema`; - const [schemaFormData] = useSessionStorage( + const [schemaFormData] = useSessionStorage( formKey, { mappings: [], @@ -117,8 +117,8 @@ export default function Page({ searchParams }: PageProps): ReactElement { const form = useForm({ mode: 'onChange', - resolver: yupResolver( - SINGLE_TABLE_SCHEMA_FORM_SCHEMA + resolver: yupResolver( + MULTI_TABLE_SCHEMA_FORM_SCHEMA ), values: schemaFormData, }); @@ -133,7 +133,7 @@ export default function Page({ searchParams }: PageProps): ReactElement { setIsClient(true); }, []); - async function onSubmit(values: SingleTableSchemaFormValues) { + async function onSubmit(values: MultiTableSchemaFormValues) { if (!account) { return; } @@ -299,7 +299,7 @@ export default function Page({ searchParams }: PageProps): ReactElement { async function createNewJob( define: DefineFormValues, connect: SingleTableConnectFormValues, - schema: SingleTableSchemaFormValues, + schema: MultiTableSchemaFormValues, accountId: string, connections: Connection[] ): Promise { diff --git a/frontend/apps/web/app/(mgmt)/[account]/new/job/schema.ts b/frontend/apps/web/app/(mgmt)/[account]/new/job/schema.ts index 55541f93a7..5542593fc8 100644 --- a/frontend/apps/web/app/(mgmt)/[account]/new/job/schema.ts +++ b/frontend/apps/web/app/(mgmt)/[account]/new/job/schema.ts @@ -251,6 +251,14 @@ export type SingleTableSchemaFormValues = Yup.InferType< typeof SINGLE_TABLE_SCHEMA_FORM_SCHEMA >; +export const MULTI_TABLE_SCHEMA_FORM_SCHEMA = Yup.object({ + mappings: Yup.array().of(JOB_MAPPING_SCHEMA).required(), + numRows: Yup.number().required().min(1), +}); +export type MultiTableSchemaFormValues = Yup.InferType< + typeof MULTI_TABLE_SCHEMA_FORM_SCHEMA +>; + export const SUBSET_FORM_SCHEMA = Yup.object({ subsets: Yup.array(SINGLE_SUBSET_FORM_SCHEMA).required(), subsetOptions: Yup.object({ diff --git a/frontend/apps/web/components/jobs/SchemaTable/SchemaTable.tsx b/frontend/apps/web/components/jobs/SchemaTable/SchemaTable.tsx index e1aab1f612..aef3c1e1dc 100644 --- a/frontend/apps/web/components/jobs/SchemaTable/SchemaTable.tsx +++ b/frontend/apps/web/components/jobs/SchemaTable/SchemaTable.tsx @@ -143,7 +143,7 @@ export function SchemaTable(props: Props): ReactElement { options={getDualListBoxOptions(schema, data)} selected={selectedItems} onChange={toggleItem} - mode={jobType === 'generate' ? 'single' : 'many'} + mode={'many'} /> diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index e66e61486a..8789d0c8eb 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -143,45 +143,15 @@ func (b *benthosBuilder) GenerateBenthosConfigs( } } + // how to handle unique constraints /////////////////////////////////////////////////////////////// - // root tables - sourceResponses, err := buildBenthosGenerateSourceConfigResponses(ctx, b.transformerclient, groupedMappings, sourceTableOpts, map[string]*dbschemas_utils.ColumnInfo{}, dependencyMap, db.Driver, sourceConnection.Id, primaryKeyMap) + sourceResponses, err := buildBenthosGenerateSourceConfigResponses(ctx, b.transformerclient, groupedMappings, sourceTableOpts, map[string]*dbschemas_utils.ColumnInfo{}, dependencyMap, db.Driver, sourceConnection.Id, td, primaryKeyMap) if err != nil { return nil, fmt.Errorf("unable to build benthos generate source config responses: %w", err) } responses = append(responses, sourceResponses...) - /////////////////////////////////////////////////////////////// - - // for _, resp := range responses { - // tableName := neosync_benthos.BuildBenthosTable(resp.TableSchema, resp.TableName) - // configs := dependencyMap[tableName] - // if len(configs) > 1 { - // // circular dependency - // for _, c := range configs { - // if c.Columns != nil && c.Columns.Exclude != nil && len(c.Columns.Exclude) > 0 { - // resp.excludeColumns = c.Columns.Exclude - // resp.DependsOn = c.DependsOn - // } else if c.Columns != nil && c.Columns.Include != nil && len(c.Columns.Include) > 0 { - // pks := primaryKeyMap[tableName] - // if len(pks) == 0 { - // return nil, fmt.Errorf("no primary keys found for table (%s). Unable to build update query", tableName) - // } - - // // config for sql update - // resp.updateConfig = c - // resp.primaryKeys = pks - // } - // } - // } else if len(configs) == 1 { - // resp.DependsOn = configs[0].DependsOn - // } else { - // return nil, fmt.Errorf("unexpected number of dependency configs") - // } - // } - /////////////////////////////////////////////////////////////// - case *mgmtv1alpha1.JobSourceOptions_Postgres, *mgmtv1alpha1.JobSourceOptions_Mysql: sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) if err != nil { @@ -388,44 +358,45 @@ func (b *benthosBuilder) GenerateBenthosConfigs( } } else if resp.Config.Input.Generate != nil { cols := buildPlainColumns(tm.Mappings) - processorConfigs := []neosync_benthos.ProcessorConfig{} - for _, pc := range resp.Processors { - processorConfigs = append(processorConfigs, *pc) - } + // processorConfigs := []neosync_benthos.ProcessorConfig{} + // for _, pc := range resp.Processors { + // processorConfigs = append(processorConfigs, *pc) + // } resp.Config.Output.Broker.Outputs = append(resp.Config.Output.Broker.Outputs, neosync_benthos.Outputs{ Fallback: []neosync_benthos.Outputs{ { // retry processor and output several times - Retry: &neosync_benthos.RetryConfig{ - InlineRetryConfig: neosync_benthos.InlineRetryConfig{ - MaxRetries: 10, - }, - Output: neosync_benthos.OutputConfig{ - Outputs: neosync_benthos.Outputs{ - PooledSqlInsert: &neosync_benthos.PooledSqlInsert{ - Driver: driver, - Dsn: dsn, - - Schema: resp.TableSchema, - Table: resp.TableName, - Columns: cols, - OnConflictDoNothing: destOpts.OnConflictDoNothing, - TruncateOnRetry: destOpts.Truncate, - - ArgsMapping: buildPlainInsertArgs(cols), - - Batching: &neosync_benthos.Batching{ - Period: "5s", - Count: 100, - }, - }, - }, - Processors: processorConfigs, + // Retry: &neosync_benthos.RetryConfig{ + // InlineRetryConfig: neosync_benthos.InlineRetryConfig{ + // MaxRetries: 10, + // }, + // Output: neosync_benthos.OutputConfig{ + // Outputs: neosync_benthos.Outputs{ + PooledSqlInsert: &neosync_benthos.PooledSqlInsert{ + Driver: driver, + Dsn: dsn, + + Schema: resp.TableSchema, + Table: resp.TableName, + Columns: cols, + OnConflictDoNothing: destOpts.OnConflictDoNothing, + TruncateOnRetry: destOpts.Truncate, + + ArgsMapping: buildPlainInsertArgs(cols), + + Batching: &neosync_benthos.Batching{ + Period: "5s", + Count: 100, }, + // }, + // }, + // Processors: processorConfigs, + // }, }, }, // kills activity depending on error + // TODO add retry here {Error: &neosync_benthos.ErrorOutputConfig{ ErrorMsg: `${! meta("fallback_error")}`, Batching: &neosync_benthos.Batching{ diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go index 67a9b315ac..890dc3fbb2 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go @@ -24,11 +24,11 @@ func buildBenthosGenerateSourceConfigResponses( columnInfo map[string]*dbschemas_utils.ColumnInfo, dependencyMap map[string][]*tabledependency.RunConfig, driver, dsnConnectionId string, + tableConstraintsMap map[string]*dbschemas_utils.TableConstraints, primaryKeyMap map[string][]string, ) ([]*BenthosConfigResponse, error) { responses := []*BenthosConfigResponse{} - // root tables for _, tableMapping := range mappings { if shared.AreAllColsNull(tableMapping.Mappings) { // skiping table as no columns are mapped @@ -75,17 +75,26 @@ func buildBenthosGenerateSourceConfigResponses( var bc *neosync_benthos.BenthosConfig if len(runConfigs) > 0 && len(runConfigs[0].DependsOn) > 0 { + columnNameMap := map[string]string{} + tableColsMaps := map[string][]string{} + + constraints := tableConstraintsMap[tableName] + for _, tc := range constraints.Constraints { + columnNameMap[fmt.Sprintf("%s.%s", tc.ForeignKey.Table, tc.ForeignKey.Column)] = tc.Column + tableColsMaps[tc.ForeignKey.Table] = append(tableColsMaps[tc.ForeignKey.Table], tc.ForeignKey.Table) + } + bc = &neosync_benthos.BenthosConfig{ StreamConfig: neosync_benthos.StreamConfig{ Input: &neosync_benthos.InputConfig{ Inputs: neosync_benthos.Inputs{ GenerateSqlSelect: &neosync_benthos.GenerateSqlSelect{ - Count: count, - Mapping: mutations, - Driver: driver, - Dsn: "${SOURCE_CONNECTION_DSN}", - // TableColumnsMap: , - // ColumnNameMap:, + Count: count, + Mapping: mutations, + Driver: driver, + Dsn: "${SOURCE_CONNECTION_DSN}", + TableColumnsMap: tableColsMaps, + ColumnNameMap: columnNameMap, }, }, }, From ab390e3772b2f514e2953e77b39255963341cc4b Mon Sep 17 00:00:00 2001 From: Alisha Date: Thu, 18 Apr 2024 14:00:04 -0700 Subject: [PATCH 07/12] frontend --- .../components/DataGenConnectionCard.tsx | 56 +++++++++++-------- .../new/job/generate/single/schema/page.tsx | 41 ++++++++------ 2 files changed, 56 insertions(+), 41 deletions(-) diff --git a/frontend/apps/web/app/(mgmt)/[account]/jobs/[id]/source/components/DataGenConnectionCard.tsx b/frontend/apps/web/app/(mgmt)/[account]/jobs/[id]/source/components/DataGenConnectionCard.tsx index 2ad7b7261e..7369281c7d 100644 --- a/frontend/apps/web/app/(mgmt)/[account]/jobs/[id]/source/components/DataGenConnectionCard.tsx +++ b/frontend/apps/web/app/(mgmt)/[account]/jobs/[id]/source/components/DataGenConnectionCard.tsx @@ -1,7 +1,7 @@ 'use client'; import { - SINGLE_TABLE_SCHEMA_FORM_SCHEMA, - SingleTableSchemaFormValues, + MULTI_TABLE_SCHEMA_FORM_SCHEMA, + MultiTableSchemaFormValues, } from '@/app/(mgmt)/[account]/new/job/schema'; import { SchemaTable } from '@/components/jobs/SchemaTable/SchemaTable'; import { getSchemaConstraintHandler } from '@/components/jobs/SchemaTable/schema-constraint-handler'; @@ -87,8 +87,8 @@ export default function DataGenConnectionCard({ jobId }: Props): ReactElement { fkSourceConnectionId ?? '' ); - const form = useForm({ - resolver: yupResolver(SINGLE_TABLE_SCHEMA_FORM_SCHEMA), + const form = useForm({ + resolver: yupResolver(MULTI_TABLE_SCHEMA_FORM_SCHEMA), values: getJobSource(data?.job), }); @@ -107,7 +107,7 @@ export default function DataGenConnectionCard({ jobId }: Props): ReactElement { return ; } - async function onSubmit(values: SingleTableSchemaFormValues) { + async function onSubmit(values: MultiTableSchemaFormValues) { const job = data?.job; if (!job) { return; @@ -182,7 +182,7 @@ export default function DataGenConnectionCard({ jobId }: Props): ReactElement { ); } -function getJobSource(job?: Job): SingleTableSchemaFormValues { +function getJobSource(job?: Job): MultiTableSchemaFormValues { if (!job) { return { mappings: [], @@ -200,7 +200,7 @@ function getJobSource(job?: Job): SingleTableSchemaFormValues { } } - const mappings: SingleTableSchemaFormValues['mappings'] = ( + const mappings: MultiTableSchemaFormValues['mappings'] = ( job.mappings ?? [] ).map((mapping) => { return { @@ -221,10 +221,31 @@ function getJobSource(job?: Job): SingleTableSchemaFormValues { async function updateJobConnection( accountId: string, job: Job, - values: SingleTableSchemaFormValues + values: MultiTableSchemaFormValues ): Promise { - const schema = values.mappings.length > 0 ? values.mappings[0].schema : null; - const table = values.mappings.length > 0 ? values.mappings[0].table : null; + const schemas = values.mappings.reduce( + (prev, curr) => { + const prevTables = prev[curr.schema] || {}; + return { + ...prev, + [curr.schema]: { ...prevTables, [curr.table]: curr.table }, + }; + }, + {} as Record> + ); + const schemaRecords = Object.entries(schemas).map(([s, tables]) => { + return new GenerateSourceSchemaOption({ + schema: s, + tables: Object.keys(tables).map( + (t) => + new GenerateSourceTableOption({ + rowCount: BigInt(values.numRows), + table: t, + }) + ), + }); + }); + const res = await fetch( `/api/accounts/${accountId}/jobs/${job.id}/source-connection`, { @@ -252,20 +273,7 @@ async function updateJobConnection( case: 'generate', value: new GenerateSourceOptions({ fkSourceConnectionId: getFkIdFromGenerateSource(job.source), - schemas: - schema && table - ? [ - new GenerateSourceSchemaOption({ - schema: schema, - tables: [ - new GenerateSourceTableOption({ - table: table, - rowCount: BigInt(values.numRows), - }), - ], - }), - ] - : [], + schemas: schemaRecords, }), }, }), diff --git a/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx b/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx index 1919176bdc..ad99fb1c82 100644 --- a/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx +++ b/frontend/apps/web/app/(mgmt)/[account]/new/job/generate/single/schema/page.tsx @@ -331,9 +331,29 @@ async function createNewJob( }), }); } - const tableSchema = - schema.mappings.length > 0 ? schema.mappings[0].schema : null; - const table = schema.mappings.length > 0 ? schema.mappings[0].table : null; + const schemas = schema.mappings.reduce( + (prev, curr) => { + const prevTables = prev[curr.schema] || {}; + return { + ...prev, + [curr.schema]: { ...prevTables, [curr.table]: curr.table }, + }; + }, + {} as Record> + ); + const schemaRecords = Object.entries(schemas).map(([s, tables]) => { + return new GenerateSourceSchemaOption({ + schema: s, + tables: Object.keys(tables).map( + (t) => + new GenerateSourceTableOption({ + rowCount: BigInt(schema.numRows), + table: t, + }) + ), + }); + }); + const body = new CreateJobRequest({ accountId, jobName: define.jobName, @@ -355,20 +375,7 @@ async function createNewJob( case: 'generate', value: new GenerateSourceOptions({ fkSourceConnectionId: connect.connectionId, - schemas: - tableSchema && table - ? [ - new GenerateSourceSchemaOption({ - schema: tableSchema, - tables: [ - new GenerateSourceTableOption({ - rowCount: BigInt(schema.numRows), - table: table, - }), - ], - }), - ] - : [], + schemas: schemaRecords, }), }, }), From 2ef96c6e88bf0ac7020396b6e1fccaa5526a6ecd Mon Sep 17 00:00:00 2001 From: Alisha Date: Fri, 19 Apr 2024 10:56:53 -0700 Subject: [PATCH 08/12] stash --- docs/protos/data/proto_docs.json | 12161 ---------------- .../client/mgmt/v1alpha1/api_key_connect.ts | 76 - .../src/client/mgmt/v1alpha1/api_key_pb.ts | 491 - .../src/client/mgmt/v1alpha1/auth_connect.ts | 89 - .../sdk/src/client/mgmt/v1alpha1/auth_pb.ts | 559 - .../mgmt/v1alpha1/connection_connect.ts | 110 - .../mgmt/v1alpha1/connection_data_connect.ts | 90 - .../mgmt/v1alpha1/connection_data_pb.ts | 1171 -- .../src/client/mgmt/v1alpha1/connection_pb.ts | 1575 -- .../src/client/mgmt/v1alpha1/job_connect.ts | 265 - .../sdk/src/client/mgmt/v1alpha1/job_pb.ts | 4384 ------ .../client/mgmt/v1alpha1/metrics_connect.ts | 39 - .../src/client/mgmt/v1alpha1/metrics_pb.ts | 398 - .../mgmt/v1alpha1/transformer_connect.ts | 107 - .../client/mgmt/v1alpha1/transformer_pb.ts | 3180 ---- .../mgmt/v1alpha1/user_account_connect.ts | 179 - .../client/mgmt/v1alpha1/user_account_pb.ts | 1659 --- worker/internal/benthos/config.go | 5 +- worker/internal/benthos/error/output_error.go | 17 +- worker/internal/benthos/utils.go | 20 +- worker/internal/benthos/utils_test.go | 4 +- .../gen-benthos-configs/benthos-builder.go | 5 +- 22 files changed, 34 insertions(+), 26550 deletions(-) delete mode 100644 docs/protos/data/proto_docs.json delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts delete mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts diff --git a/docs/protos/data/proto_docs.json b/docs/protos/data/proto_docs.json deleted file mode 100644 index 1b0439c611..0000000000 --- a/docs/protos/data/proto_docs.json +++ /dev/null @@ -1,12161 +0,0 @@ -{ - "files": [ - { - "name": "mgmt/v1alpha1/api_key.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": false, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [], - "extensions": [], - "messages": [ - { - "name": "AccountApiKey", - "longName": "AccountApiKey", - "fullName": "mgmt.v1alpha1.AccountApiKey", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "The friendly name of the API Key", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_by_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_by_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "key_value", - "description": "key_value is only returned on initial creation or when it is regenerated", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_key_value", - "defaultValue": "" - }, - { - "name": "user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "expires_at", - "description": "The timestamp of what the API key expires and will not longer be usable.", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateAccountApiKeyRequest", - "longName": "CreateAccountApiKeyRequest", - "fullName": "mgmt.v1alpha1.CreateAccountApiKeyRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "expires_at", - "description": "Validate between now and one year: now \u003c x \u003c 365 days", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateAccountApiKeyResponse", - "longName": "CreateAccountApiKeyResponse", - "fullName": "mgmt.v1alpha1.CreateAccountApiKeyResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "api_key", - "description": "", - "label": "", - "type": "AccountApiKey", - "longType": "AccountApiKey", - "fullType": "mgmt.v1alpha1.AccountApiKey", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteAccountApiKeyRequest", - "longName": "DeleteAccountApiKeyRequest", - "fullName": "mgmt.v1alpha1.DeleteAccountApiKeyRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteAccountApiKeyResponse", - "longName": "DeleteAccountApiKeyResponse", - "fullName": "mgmt.v1alpha1.DeleteAccountApiKeyResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetAccountApiKeyRequest", - "longName": "GetAccountApiKeyRequest", - "fullName": "mgmt.v1alpha1.GetAccountApiKeyRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountApiKeyResponse", - "longName": "GetAccountApiKeyResponse", - "fullName": "mgmt.v1alpha1.GetAccountApiKeyResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "api_key", - "description": "", - "label": "", - "type": "AccountApiKey", - "longType": "AccountApiKey", - "fullType": "mgmt.v1alpha1.AccountApiKey", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountApiKeysRequest", - "longName": "GetAccountApiKeysRequest", - "fullName": "mgmt.v1alpha1.GetAccountApiKeysRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountApiKeysResponse", - "longName": "GetAccountApiKeysResponse", - "fullName": "mgmt.v1alpha1.GetAccountApiKeysResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "api_keys", - "description": "", - "label": "repeated", - "type": "AccountApiKey", - "longType": "AccountApiKey", - "fullType": "mgmt.v1alpha1.AccountApiKey", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RegenerateAccountApiKeyRequest", - "longName": "RegenerateAccountApiKeyRequest", - "fullName": "mgmt.v1alpha1.RegenerateAccountApiKeyRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "expires_at", - "description": "Validate between now and one year: now \u003c x \u003c 365 days", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RegenerateAccountApiKeyResponse", - "longName": "RegenerateAccountApiKeyResponse", - "fullName": "mgmt.v1alpha1.RegenerateAccountApiKeyResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "api_key", - "description": "", - "label": "", - "type": "AccountApiKey", - "longType": "AccountApiKey", - "fullType": "mgmt.v1alpha1.AccountApiKey", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "ApiKeyService", - "longName": "ApiKeyService", - "fullName": "mgmt.v1alpha1.ApiKeyService", - "description": "Service that manages the lifecycle of API Keys that are associated with a specific Account.", - "methods": [ - { - "name": "GetAccountApiKeys", - "description": "Retrieves a list of Account API Keys", - "requestType": "GetAccountApiKeysRequest", - "requestLongType": "GetAccountApiKeysRequest", - "requestFullType": "mgmt.v1alpha1.GetAccountApiKeysRequest", - "requestStreaming": false, - "responseType": "GetAccountApiKeysResponse", - "responseLongType": "GetAccountApiKeysResponse", - "responseFullType": "mgmt.v1alpha1.GetAccountApiKeysResponse", - "responseStreaming": false - }, - { - "name": "GetAccountApiKey", - "description": "Retrieves a single API Key", - "requestType": "GetAccountApiKeyRequest", - "requestLongType": "GetAccountApiKeyRequest", - "requestFullType": "mgmt.v1alpha1.GetAccountApiKeyRequest", - "requestStreaming": false, - "responseType": "GetAccountApiKeyResponse", - "responseLongType": "GetAccountApiKeyResponse", - "responseFullType": "mgmt.v1alpha1.GetAccountApiKeyResponse", - "responseStreaming": false - }, - { - "name": "CreateAccountApiKey", - "description": "Creates a single API Key\nThis method will return the decrypted contents of the API key", - "requestType": "CreateAccountApiKeyRequest", - "requestLongType": "CreateAccountApiKeyRequest", - "requestFullType": "mgmt.v1alpha1.CreateAccountApiKeyRequest", - "requestStreaming": false, - "responseType": "CreateAccountApiKeyResponse", - "responseLongType": "CreateAccountApiKeyResponse", - "responseFullType": "mgmt.v1alpha1.CreateAccountApiKeyResponse", - "responseStreaming": false - }, - { - "name": "RegenerateAccountApiKey", - "description": "Regenerates a single API Key with a new expiration time\nThis method will return the decrypted contents of the API key", - "requestType": "RegenerateAccountApiKeyRequest", - "requestLongType": "RegenerateAccountApiKeyRequest", - "requestFullType": "mgmt.v1alpha1.RegenerateAccountApiKeyRequest", - "requestStreaming": false, - "responseType": "RegenerateAccountApiKeyResponse", - "responseLongType": "RegenerateAccountApiKeyResponse", - "responseFullType": "mgmt.v1alpha1.RegenerateAccountApiKeyResponse", - "responseStreaming": false - }, - { - "name": "DeleteAccountApiKey", - "description": "Deletes an API Key from the system.", - "requestType": "DeleteAccountApiKeyRequest", - "requestLongType": "DeleteAccountApiKeyRequest", - "requestFullType": "mgmt.v1alpha1.DeleteAccountApiKeyRequest", - "requestStreaming": false, - "responseType": "DeleteAccountApiKeyResponse", - "responseLongType": "DeleteAccountApiKeyResponse", - "responseFullType": "mgmt.v1alpha1.DeleteAccountApiKeyResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/auth.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": false, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [], - "extensions": [], - "messages": [ - { - "name": "AccessToken", - "longName": "AccessToken", - "fullName": "mgmt.v1alpha1.AccessToken", - "description": "A decoded representation of an Access token from the backing auth server", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "access_token", - "description": "The access token that will be provided in subsequent requests to provide authenticated access to the Api", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "refresh_token", - "description": "Token that can be used to retrieve a refreshed access token.\nWill not be provided if the offline_access scope is not provided in the initial login flow.", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_refresh_token", - "defaultValue": "" - }, - { - "name": "expires_in", - "description": "Relative time in seconds that the access token will expire. Combine with the current time to get the expires_at time.", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "scope", - "description": "The scopes that the access token have", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "id_token", - "description": "The identity token of the authenticated user", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_id_token", - "defaultValue": "" - }, - { - "name": "token_type", - "description": "The token type. For JWTs, this will be `Bearer`", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CheckTokenRequest", - "longName": "CheckTokenRequest", - "fullName": "mgmt.v1alpha1.CheckTokenRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "CheckTokenResponse", - "longName": "CheckTokenResponse", - "fullName": "mgmt.v1alpha1.CheckTokenResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetAuthStatusRequest", - "longName": "GetAuthStatusRequest", - "fullName": "mgmt.v1alpha1.GetAuthStatusRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetAuthStatusResponse", - "longName": "GetAuthStatusResponse", - "fullName": "mgmt.v1alpha1.GetAuthStatusResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "is_enabled", - "description": "Whether or not the server has authentication enabled.\nThis tells the client if it is expected to send access tokens.", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAuthorizeUrlRequest", - "longName": "GetAuthorizeUrlRequest", - "fullName": "mgmt.v1alpha1.GetAuthorizeUrlRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "state", - "description": "The state that's generated by the client that is passed along to prevent tampering", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "redirect_uri", - "description": "The redirect uri that the client will be redirected back to during the auth request", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "scope", - "description": "The scopes the client is requesting as a part of the oauth login request", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAuthorizeUrlResponse", - "longName": "GetAuthorizeUrlResponse", - "fullName": "mgmt.v1alpha1.GetAuthorizeUrlResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "url", - "description": "The generated url that is the client will be redirected to during the Oauth flow", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetCliIssuerRequest", - "longName": "GetCliIssuerRequest", - "fullName": "mgmt.v1alpha1.GetCliIssuerRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetCliIssuerResponse", - "longName": "GetCliIssuerResponse", - "fullName": "mgmt.v1alpha1.GetCliIssuerResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "issuer_url", - "description": "The backing authentication issuer url", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "audience", - "description": "The audience that will be used in the access token. This corresponds to the \"aud\" claim", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "LoginCliRequest", - "longName": "LoginCliRequest", - "fullName": "mgmt.v1alpha1.LoginCliRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "code", - "description": "The oauth code", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "redirect_uri", - "description": "The oauth redirect uri that the client uses during the oauth request", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "LoginCliResponse", - "longName": "LoginCliResponse", - "fullName": "mgmt.v1alpha1.LoginCliResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "access_token", - "description": "The access token that is returned on successful login", - "label": "", - "type": "AccessToken", - "longType": "AccessToken", - "fullType": "mgmt.v1alpha1.AccessToken", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RefreshCliRequest", - "longName": "RefreshCliRequest", - "fullName": "mgmt.v1alpha1.RefreshCliRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "refresh_token", - "description": "The token used to retrieve a new access token.", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RefreshCliResponse", - "longName": "RefreshCliResponse", - "fullName": "mgmt.v1alpha1.RefreshCliResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "access_token", - "description": "The access token that is returned on successful refresh", - "label": "", - "type": "AccessToken", - "longType": "AccessToken", - "fullType": "mgmt.v1alpha1.AccessToken", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "AuthService", - "longName": "AuthService", - "fullName": "mgmt.v1alpha1.AuthService", - "description": "Service that handles generic Authentication for Neosync\nToday this is mostly used by the CLI to receive authentication information", - "methods": [ - { - "name": "LoginCli", - "description": "Used by the CLI to login to Neosync with OAuth.", - "requestType": "LoginCliRequest", - "requestLongType": "LoginCliRequest", - "requestFullType": "mgmt.v1alpha1.LoginCliRequest", - "requestStreaming": false, - "responseType": "LoginCliResponse", - "responseLongType": "LoginCliResponse", - "responseFullType": "mgmt.v1alpha1.LoginCliResponse", - "responseStreaming": false - }, - { - "name": "RefreshCli", - "description": "Used by the CLI to refresh an expired Neosync accesss token.\nThis should only be used if an access token was previously retrieved from the `LoginCli` or `RefreshCli` methods.", - "requestType": "RefreshCliRequest", - "requestLongType": "RefreshCliRequest", - "requestFullType": "mgmt.v1alpha1.RefreshCliRequest", - "requestStreaming": false, - "responseType": "RefreshCliResponse", - "responseLongType": "RefreshCliResponse", - "responseFullType": "mgmt.v1alpha1.RefreshCliResponse", - "responseStreaming": false - }, - { - "name": "CheckToken", - "description": "Empty endpoint to simply check if the provided access token is valid", - "requestType": "CheckTokenRequest", - "requestLongType": "CheckTokenRequest", - "requestFullType": "mgmt.v1alpha1.CheckTokenRequest", - "requestStreaming": false, - "responseType": "CheckTokenResponse", - "responseLongType": "CheckTokenResponse", - "responseFullType": "mgmt.v1alpha1.CheckTokenResponse", - "responseStreaming": false - }, - { - "name": "GetCliIssuer", - "description": "Used by the CLI to retrieve Auth Issuer information\n@deprecated", - "requestType": "GetCliIssuerRequest", - "requestLongType": "GetCliIssuerRequest", - "requestFullType": "mgmt.v1alpha1.GetCliIssuerRequest", - "requestStreaming": false, - "responseType": "GetCliIssuerResponse", - "responseLongType": "GetCliIssuerResponse", - "responseFullType": "mgmt.v1alpha1.GetCliIssuerResponse", - "responseStreaming": false - }, - { - "name": "GetAuthorizeUrl", - "description": "Used by the CLI to retrieve an Authorize URL for use with OAuth login.", - "requestType": "GetAuthorizeUrlRequest", - "requestLongType": "GetAuthorizeUrlRequest", - "requestFullType": "mgmt.v1alpha1.GetAuthorizeUrlRequest", - "requestStreaming": false, - "responseType": "GetAuthorizeUrlResponse", - "responseLongType": "GetAuthorizeUrlResponse", - "responseFullType": "mgmt.v1alpha1.GetAuthorizeUrlResponse", - "responseStreaming": false - }, - { - "name": "GetAuthStatus", - "description": "Returns the auth status of the API server. Whether or not the backend has authentication enabled.\nThis is used by clients to make decisions on whether or not they should send access tokens to the API.", - "requestType": "GetAuthStatusRequest", - "requestLongType": "GetAuthStatusRequest", - "requestFullType": "mgmt.v1alpha1.GetAuthStatusRequest", - "requestStreaming": false, - "responseType": "GetAuthStatusResponse", - "responseLongType": "GetAuthStatusResponse", - "responseFullType": "mgmt.v1alpha1.GetAuthStatusResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/connection.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": false, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [], - "extensions": [], - "messages": [ - { - "name": "AwsS3ConnectionConfig", - "longName": "AwsS3ConnectionConfig", - "fullName": "mgmt.v1alpha1.AwsS3ConnectionConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "bucket_arn", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "", - "options": { - "deprecated": true - } - }, - { - "name": "path_prefix", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_path_prefix", - "defaultValue": "" - }, - { - "name": "credentials", - "description": "", - "label": "optional", - "type": "AwsS3Credentials", - "longType": "AwsS3Credentials", - "fullType": "mgmt.v1alpha1.AwsS3Credentials", - "ismap": false, - "isoneof": true, - "oneofdecl": "_credentials", - "defaultValue": "" - }, - { - "name": "region", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_region", - "defaultValue": "" - }, - { - "name": "endpoint", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_endpoint", - "defaultValue": "" - }, - { - "name": "bucket", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AwsS3Credentials", - "longName": "AwsS3Credentials", - "fullName": "mgmt.v1alpha1.AwsS3Credentials", - "description": "S3 Credentials that are used by the worker process.\nNote: this may be optionally provided if the worker that is being hosted has environment credentials to the S3 bucket instead.", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "profile", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_profile", - "defaultValue": "" - }, - { - "name": "access_key_id", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_access_key_id", - "defaultValue": "" - }, - { - "name": "secret_access_key", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_secret_access_key", - "defaultValue": "" - }, - { - "name": "session_token", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_session_token", - "defaultValue": "" - }, - { - "name": "from_ec2_role", - "description": "", - "label": "optional", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": true, - "oneofdecl": "_from_ec2_role", - "defaultValue": "" - }, - { - "name": "role_arn", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_role_arn", - "defaultValue": "" - }, - { - "name": "role_external_id", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_role_external_id", - "defaultValue": "" - } - ] - }, - { - "name": "CheckConnectionConfigRequest", - "longName": "CheckConnectionConfigRequest", - "fullName": "mgmt.v1alpha1.CheckConnectionConfigRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_config", - "description": "", - "label": "", - "type": "ConnectionConfig", - "longType": "ConnectionConfig", - "fullType": "mgmt.v1alpha1.ConnectionConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CheckConnectionConfigResponse", - "longName": "CheckConnectionConfigResponse", - "fullName": "mgmt.v1alpha1.CheckConnectionConfigResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "is_connected", - "description": "Whether or not the API was able to ping the connection", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_error", - "description": "This is the error that was received if the API was unable to connect", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_connection_error", - "defaultValue": "" - }, - { - "name": "privileges", - "description": "", - "label": "repeated", - "type": "ConnectionRolePrivilege", - "longType": "ConnectionRolePrivilege", - "fullType": "mgmt.v1alpha1.ConnectionRolePrivilege", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CheckSqlQueryRequest", - "longName": "CheckSqlQueryRequest", - "fullName": "mgmt.v1alpha1.CheckSqlQueryRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "The connection id that the query will be checked against", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "query", - "description": "The full query that will be run through a PREPARE statement", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CheckSqlQueryResponse", - "longName": "CheckSqlQueryResponse", - "fullName": "mgmt.v1alpha1.CheckSqlQueryResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "is_valid", - "description": "The query is run through PREPARE. Returns valid if it correctly compiled", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "erorr_message", - "description": "The error message returned by the sql client if the prepare did not return successfully", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_erorr_message", - "defaultValue": "" - } - ] - }, - { - "name": "Connection", - "longName": "Connection", - "fullName": "mgmt.v1alpha1.Connection", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_config", - "description": "", - "label": "", - "type": "ConnectionConfig", - "longType": "ConnectionConfig", - "fullType": "mgmt.v1alpha1.ConnectionConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_by_user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_by_user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ConnectionConfig", - "longName": "ConnectionConfig", - "fullName": "mgmt.v1alpha1.ConnectionConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "pg_config", - "description": "", - "label": "", - "type": "PostgresConnectionConfig", - "longType": "PostgresConnectionConfig", - "fullType": "mgmt.v1alpha1.PostgresConnectionConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "aws_s3_config", - "description": "", - "label": "", - "type": "AwsS3ConnectionConfig", - "longType": "AwsS3ConnectionConfig", - "fullType": "mgmt.v1alpha1.AwsS3ConnectionConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "mysql_config", - "description": "", - "label": "", - "type": "MysqlConnectionConfig", - "longType": "MysqlConnectionConfig", - "fullType": "mgmt.v1alpha1.MysqlConnectionConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "local_dir_config", - "description": "Configures a connection to a directory available on the local file system", - "label": "", - "type": "LocalDirectoryConnectionConfig", - "longType": "LocalDirectoryConnectionConfig", - "fullType": "mgmt.v1alpha1.LocalDirectoryConnectionConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "ConnectionRolePrivilege", - "longName": "ConnectionRolePrivilege", - "fullName": "mgmt.v1alpha1.ConnectionRolePrivilege", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "grantee", - "description": "The role that was given the permissions", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schema", - "description": "The database schema. Ex: public", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table", - "description": "The name of the table in the schema", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "privilege_type", - "description": "The privileges given to that role", - "label": "repeated", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateConnectionRequest", - "longName": "CreateConnectionRequest", - "fullName": "mgmt.v1alpha1.CreateConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "The friendly name of the connection", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_config", - "description": "", - "label": "", - "type": "ConnectionConfig", - "longType": "ConnectionConfig", - "fullType": "mgmt.v1alpha1.ConnectionConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateConnectionResponse", - "longName": "CreateConnectionResponse", - "fullName": "mgmt.v1alpha1.CreateConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection", - "description": "", - "label": "", - "type": "Connection", - "longType": "Connection", - "fullType": "mgmt.v1alpha1.Connection", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteConnectionRequest", - "longName": "DeleteConnectionRequest", - "fullName": "mgmt.v1alpha1.DeleteConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteConnectionResponse", - "longName": "DeleteConnectionResponse", - "fullName": "mgmt.v1alpha1.DeleteConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetConnectionRequest", - "longName": "GetConnectionRequest", - "fullName": "mgmt.v1alpha1.GetConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionResponse", - "longName": "GetConnectionResponse", - "fullName": "mgmt.v1alpha1.GetConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection", - "description": "", - "label": "", - "type": "Connection", - "longType": "Connection", - "fullType": "mgmt.v1alpha1.Connection", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionsRequest", - "longName": "GetConnectionsRequest", - "fullName": "mgmt.v1alpha1.GetConnectionsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionsResponse", - "longName": "GetConnectionsResponse", - "fullName": "mgmt.v1alpha1.GetConnectionsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connections", - "description": "", - "label": "repeated", - "type": "Connection", - "longType": "Connection", - "fullType": "mgmt.v1alpha1.Connection", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsConnectionNameAvailableRequest", - "longName": "IsConnectionNameAvailableRequest", - "fullName": "mgmt.v1alpha1.IsConnectionNameAvailableRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsConnectionNameAvailableResponse", - "longName": "IsConnectionNameAvailableResponse", - "fullName": "mgmt.v1alpha1.IsConnectionNameAvailableResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "is_available", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "LocalDirectoryConnectionConfig", - "longName": "LocalDirectoryConnectionConfig", - "fullName": "mgmt.v1alpha1.LocalDirectoryConnectionConfig", - "description": "Configures a connection to a directory available on the local file system", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "path", - "description": "The absolute path to a directory that is available on the local file system to the API and Worker nodes", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlConnection", - "longName": "MysqlConnection", - "fullName": "mgmt.v1alpha1.MysqlConnection", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "user", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "pass", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "protocol", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "host", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "port", - "description": "", - "label": "", - "type": "int32", - "longType": "int32", - "fullType": "int32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlConnectionConfig", - "longName": "MysqlConnectionConfig", - "fullName": "mgmt.v1alpha1.MysqlConnectionConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "url", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "connection_config", - "defaultValue": "" - }, - { - "name": "connection", - "description": "", - "label": "", - "type": "MysqlConnection", - "longType": "MysqlConnection", - "fullType": "mgmt.v1alpha1.MysqlConnection", - "ismap": false, - "isoneof": true, - "oneofdecl": "connection_config", - "defaultValue": "" - }, - { - "name": "tunnel", - "description": "Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet", - "label": "", - "type": "SSHTunnel", - "longType": "SSHTunnel", - "fullType": "mgmt.v1alpha1.SSHTunnel", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_options", - "description": "", - "label": "", - "type": "SqlConnectionOptions", - "longType": "SqlConnectionOptions", - "fullType": "mgmt.v1alpha1.SqlConnectionOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresConnection", - "longName": "PostgresConnection", - "fullName": "mgmt.v1alpha1.PostgresConnection", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "host", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "port", - "description": "", - "label": "", - "type": "int32", - "longType": "int32", - "fullType": "int32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "user", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "pass", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "ssl_mode", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_ssl_mode", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresConnectionConfig", - "longName": "PostgresConnectionConfig", - "fullName": "mgmt.v1alpha1.PostgresConnectionConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "url", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "connection_config", - "defaultValue": "" - }, - { - "name": "connection", - "description": "", - "label": "", - "type": "PostgresConnection", - "longType": "PostgresConnection", - "fullType": "mgmt.v1alpha1.PostgresConnection", - "ismap": false, - "isoneof": true, - "oneofdecl": "connection_config", - "defaultValue": "" - }, - { - "name": "tunnel", - "description": "Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet", - "label": "", - "type": "SSHTunnel", - "longType": "SSHTunnel", - "fullType": "mgmt.v1alpha1.SSHTunnel", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_options", - "description": "", - "label": "", - "type": "SqlConnectionOptions", - "longType": "SqlConnectionOptions", - "fullType": "mgmt.v1alpha1.SqlConnectionOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SSHAuthentication", - "longName": "SSHAuthentication", - "fullName": "mgmt.v1alpha1.SSHAuthentication", - "description": "SSH Authentication", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "passphrase", - "description": "", - "label": "", - "type": "SSHPassphrase", - "longType": "SSHPassphrase", - "fullType": "mgmt.v1alpha1.SSHPassphrase", - "ismap": false, - "isoneof": true, - "oneofdecl": "auth_config", - "defaultValue": "" - }, - { - "name": "private_key", - "description": "", - "label": "", - "type": "SSHPrivateKey", - "longType": "SSHPrivateKey", - "fullType": "mgmt.v1alpha1.SSHPrivateKey", - "ismap": false, - "isoneof": true, - "oneofdecl": "auth_config", - "defaultValue": "" - } - ] - }, - { - "name": "SSHPassphrase", - "longName": "SSHPassphrase", - "fullName": "mgmt.v1alpha1.SSHPassphrase", - "description": "Contains the configuration needed to retrieve the SSH passphrase for the tunnel", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "value", - "description": "The password to be used to connect to the SSH server\n\neventually we can expand this to allow pulling from other sources.", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SSHPrivateKey", - "longName": "SSHPrivateKey", - "fullName": "mgmt.v1alpha1.SSHPrivateKey", - "description": "Contains the configuration needed to retrieve the SSH private key for the tunnel", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "value", - "description": "The private key in PEM format.", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "passphrase", - "description": "If the private key is encrypted, this value should decrypt it.", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_passphrase", - "defaultValue": "" - } - ] - }, - { - "name": "SSHTunnel", - "longName": "SSHTunnel", - "fullName": "mgmt.v1alpha1.SSHTunnel", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "host", - "description": "The host of the SSH server", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "port", - "description": "The port of the SSH server, typically 22", - "label": "", - "type": "int32", - "longType": "int32", - "fullType": "int32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "user", - "description": "The user that will be used to authenticate", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "known_host_public_key", - "description": "Optionally provide the public key of the known host of the SSH tunnel that we are connecting to.\nIf this is not provided, the server will blindly connect to the host with the given credentials.\nThis is not recommended for production use!", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_known_host_public_key", - "defaultValue": "" - }, - { - "name": "authentication", - "description": "Provide the authentication required to successfully connect to the SSH server for tunneling", - "label": "", - "type": "SSHAuthentication", - "longType": "SSHAuthentication", - "fullType": "mgmt.v1alpha1.SSHAuthentication", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SqlConnectionOptions", - "longName": "SqlConnectionOptions", - "fullName": "mgmt.v1alpha1.SqlConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "max_connection_limit", - "description": "", - "label": "optional", - "type": "int32", - "longType": "int32", - "fullType": "int32", - "ismap": false, - "isoneof": true, - "oneofdecl": "_max_connection_limit", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateConnectionRequest", - "longName": "UpdateConnectionRequest", - "fullName": "mgmt.v1alpha1.UpdateConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_config", - "description": "", - "label": "", - "type": "ConnectionConfig", - "longType": "ConnectionConfig", - "fullType": "mgmt.v1alpha1.ConnectionConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateConnectionResponse", - "longName": "UpdateConnectionResponse", - "fullName": "mgmt.v1alpha1.UpdateConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection", - "description": "", - "label": "", - "type": "Connection", - "longType": "Connection", - "fullType": "mgmt.v1alpha1.Connection", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "ConnectionService", - "longName": "ConnectionService", - "fullName": "mgmt.v1alpha1.ConnectionService", - "description": "Service for managing datasource connections.\nThis is a primary data model in Neosync and is used in reference when hooking up Jobs to synchronize and generate data.", - "methods": [ - { - "name": "GetConnections", - "description": "Returns a list of connections associated with the account", - "requestType": "GetConnectionsRequest", - "requestLongType": "GetConnectionsRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionsRequest", - "requestStreaming": false, - "responseType": "GetConnectionsResponse", - "responseLongType": "GetConnectionsResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionsResponse", - "responseStreaming": false - }, - { - "name": "GetConnection", - "description": "Returns a single connection", - "requestType": "GetConnectionRequest", - "requestLongType": "GetConnectionRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionRequest", - "requestStreaming": false, - "responseType": "GetConnectionResponse", - "responseLongType": "GetConnectionResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionResponse", - "responseStreaming": false - }, - { - "name": "CreateConnection", - "description": "Creates a new connection", - "requestType": "CreateConnectionRequest", - "requestLongType": "CreateConnectionRequest", - "requestFullType": "mgmt.v1alpha1.CreateConnectionRequest", - "requestStreaming": false, - "responseType": "CreateConnectionResponse", - "responseLongType": "CreateConnectionResponse", - "responseFullType": "mgmt.v1alpha1.CreateConnectionResponse", - "responseStreaming": false - }, - { - "name": "UpdateConnection", - "description": "Updates an existing connection", - "requestType": "UpdateConnectionRequest", - "requestLongType": "UpdateConnectionRequest", - "requestFullType": "mgmt.v1alpha1.UpdateConnectionRequest", - "requestStreaming": false, - "responseType": "UpdateConnectionResponse", - "responseLongType": "UpdateConnectionResponse", - "responseFullType": "mgmt.v1alpha1.UpdateConnectionResponse", - "responseStreaming": false - }, - { - "name": "DeleteConnection", - "description": "Removes a connection from the system.", - "requestType": "DeleteConnectionRequest", - "requestLongType": "DeleteConnectionRequest", - "requestFullType": "mgmt.v1alpha1.DeleteConnectionRequest", - "requestStreaming": false, - "responseType": "DeleteConnectionResponse", - "responseLongType": "DeleteConnectionResponse", - "responseFullType": "mgmt.v1alpha1.DeleteConnectionResponse", - "responseStreaming": false - }, - { - "name": "IsConnectionNameAvailable", - "description": "Connections have friendly names, this method checks if the requested name is available in the system based on the account", - "requestType": "IsConnectionNameAvailableRequest", - "requestLongType": "IsConnectionNameAvailableRequest", - "requestFullType": "mgmt.v1alpha1.IsConnectionNameAvailableRequest", - "requestStreaming": false, - "responseType": "IsConnectionNameAvailableResponse", - "responseLongType": "IsConnectionNameAvailableResponse", - "responseFullType": "mgmt.v1alpha1.IsConnectionNameAvailableResponse", - "responseStreaming": false - }, - { - "name": "CheckConnectionConfig", - "description": "Checks if the connection config is connectable by the backend.\nUsed mostly to verify that a connection is valid prior to creating a Connection object.", - "requestType": "CheckConnectionConfigRequest", - "requestLongType": "CheckConnectionConfigRequest", - "requestFullType": "mgmt.v1alpha1.CheckConnectionConfigRequest", - "requestStreaming": false, - "responseType": "CheckConnectionConfigResponse", - "responseLongType": "CheckConnectionConfigResponse", - "responseFullType": "mgmt.v1alpha1.CheckConnectionConfigResponse", - "responseStreaming": false - }, - { - "name": "CheckSqlQuery", - "description": "Checks a constructed SQL query against a sql-based connection to see if it's valid based on that connection's data schema\nThis is useful when constructing subsets to see if the WHERE clause is correct", - "requestType": "CheckSqlQueryRequest", - "requestLongType": "CheckSqlQueryRequest", - "requestFullType": "mgmt.v1alpha1.CheckSqlQueryRequest", - "requestStreaming": false, - "responseType": "CheckSqlQueryResponse", - "responseLongType": "CheckSqlQueryResponse", - "responseFullType": "mgmt.v1alpha1.CheckSqlQueryResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/connection_data.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": false, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [], - "extensions": [], - "messages": [ - { - "name": "AwsS3SchemaConfig", - "longName": "AwsS3SchemaConfig", - "fullName": "mgmt.v1alpha1.AwsS3SchemaConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - }, - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - } - ] - }, - { - "name": "AwsS3StreamConfig", - "longName": "AwsS3StreamConfig", - "fullName": "mgmt.v1alpha1.AwsS3StreamConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - }, - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - } - ] - }, - { - "name": "ConnectionSchemaConfig", - "longName": "ConnectionSchemaConfig", - "fullName": "mgmt.v1alpha1.ConnectionSchemaConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "pg_config", - "description": "", - "label": "", - "type": "PostgresSchemaConfig", - "longType": "PostgresSchemaConfig", - "fullType": "mgmt.v1alpha1.PostgresSchemaConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "aws_s3_config", - "description": "", - "label": "", - "type": "AwsS3SchemaConfig", - "longType": "AwsS3SchemaConfig", - "fullType": "mgmt.v1alpha1.AwsS3SchemaConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "mysql_config", - "description": "", - "label": "", - "type": "MysqlSchemaConfig", - "longType": "MysqlSchemaConfig", - "fullType": "mgmt.v1alpha1.MysqlSchemaConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "ConnectionStreamConfig", - "longName": "ConnectionStreamConfig", - "fullName": "mgmt.v1alpha1.ConnectionStreamConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "pg_config", - "description": "", - "label": "", - "type": "PostgresStreamConfig", - "longType": "PostgresStreamConfig", - "fullType": "mgmt.v1alpha1.PostgresStreamConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "aws_s3_config", - "description": "", - "label": "", - "type": "AwsS3StreamConfig", - "longType": "AwsS3StreamConfig", - "fullType": "mgmt.v1alpha1.AwsS3StreamConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "mysql_config", - "description": "", - "label": "", - "type": "MysqlStreamConfig", - "longType": "MysqlStreamConfig", - "fullType": "mgmt.v1alpha1.MysqlStreamConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "DatabaseColumn", - "longName": "DatabaseColumn", - "fullName": "mgmt.v1alpha1.DatabaseColumn", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "The database schema. Ex: public", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table", - "description": "The name of the table in the schema", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "column", - "description": "The name of the column", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "data_type", - "description": "The datatype of the column", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "is_nullable", - "description": "The isNullable Flag of the column", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ForeignConstraint", - "longName": "ForeignConstraint", - "fullName": "mgmt.v1alpha1.ForeignConstraint", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "column", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "is_nullable", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "foreign_key", - "description": "", - "label": "", - "type": "ForeignKey", - "longType": "ForeignKey", - "fullType": "mgmt.v1alpha1.ForeignKey", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ForeignConstraintTables", - "longName": "ForeignConstraintTables", - "fullName": "mgmt.v1alpha1.ForeignConstraintTables", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "constraints", - "description": "", - "label": "repeated", - "type": "ForeignConstraint", - "longType": "ForeignConstraint", - "fullType": "mgmt.v1alpha1.ForeignConstraint", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ForeignKey", - "longName": "ForeignKey", - "fullName": "mgmt.v1alpha1.ForeignKey", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "column", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionDataStreamRequest", - "longName": "GetConnectionDataStreamRequest", - "fullName": "mgmt.v1alpha1.GetConnectionDataStreamRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "stream_config", - "description": "", - "label": "", - "type": "ConnectionStreamConfig", - "longType": "ConnectionStreamConfig", - "fullType": "mgmt.v1alpha1.ConnectionStreamConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionDataStreamResponse", - "longName": "GetConnectionDataStreamResponse", - "fullName": "mgmt.v1alpha1.GetConnectionDataStreamResponse", - "description": "Each stream response is a single row in the requested schema and table", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "row", - "description": "A map of column name to the bytes value of the data that was found for that column and row", - "label": "repeated", - "type": "RowEntry", - "longType": "GetConnectionDataStreamResponse.RowEntry", - "fullType": "mgmt.v1alpha1.GetConnectionDataStreamResponse.RowEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RowEntry", - "longName": "GetConnectionDataStreamResponse.RowEntry", - "fullName": "mgmt.v1alpha1.GetConnectionDataStreamResponse.RowEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "bytes", - "longType": "bytes", - "fullType": "bytes", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionForeignConstraintsRequest", - "longName": "GetConnectionForeignConstraintsRequest", - "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionForeignConstraintsResponse", - "longName": "GetConnectionForeignConstraintsResponse", - "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse", - "description": "Dependency constraints for a specific table", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table_constraints", - "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and the list of tables that it depends on, also `\u003cschema\u003e.\u003ctable\u003e` format.", - "label": "repeated", - "type": "TableConstraintsEntry", - "longType": "GetConnectionForeignConstraintsResponse.TableConstraintsEntry", - "fullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse.TableConstraintsEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TableConstraintsEntry", - "longName": "GetConnectionForeignConstraintsResponse.TableConstraintsEntry", - "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse.TableConstraintsEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "ForeignConstraintTables", - "longType": "ForeignConstraintTables", - "fullType": "mgmt.v1alpha1.ForeignConstraintTables", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionInitStatementsRequest", - "longName": "GetConnectionInitStatementsRequest", - "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "options", - "description": "", - "label": "", - "type": "InitStatementOptions", - "longType": "InitStatementOptions", - "fullType": "mgmt.v1alpha1.InitStatementOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionInitStatementsResponse", - "longName": "GetConnectionInitStatementsResponse", - "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse", - "description": "Init statement for a specific table", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table_init_statements", - "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the table init statement.", - "label": "repeated", - "type": "TableInitStatementsEntry", - "longType": "GetConnectionInitStatementsResponse.TableInitStatementsEntry", - "fullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableInitStatementsEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table_truncate_statements", - "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the table truncate statement.", - "label": "repeated", - "type": "TableTruncateStatementsEntry", - "longType": "GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", - "fullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TableInitStatementsEntry", - "longName": "GetConnectionInitStatementsResponse.TableInitStatementsEntry", - "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableInitStatementsEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TableTruncateStatementsEntry", - "longName": "GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", - "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionPrimaryConstraintsRequest", - "longName": "GetConnectionPrimaryConstraintsRequest", - "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest", - "description": "Primary constraints for a specific table", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionPrimaryConstraintsResponse", - "longName": "GetConnectionPrimaryConstraintsResponse", - "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table_constraints", - "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the primary constraint", - "label": "repeated", - "type": "TableConstraintsEntry", - "longType": "GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", - "fullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TableConstraintsEntry", - "longName": "GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", - "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "PrimaryConstraint", - "longType": "PrimaryConstraint", - "fullType": "mgmt.v1alpha1.PrimaryConstraint", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionSchemaRequest", - "longName": "GetConnectionSchemaRequest", - "fullName": "mgmt.v1alpha1.GetConnectionSchemaRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schema_config", - "description": "", - "label": "", - "type": "ConnectionSchemaConfig", - "longType": "ConnectionSchemaConfig", - "fullType": "mgmt.v1alpha1.ConnectionSchemaConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionSchemaResponse", - "longName": "GetConnectionSchemaResponse", - "fullName": "mgmt.v1alpha1.GetConnectionSchemaResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schemas", - "description": "", - "label": "repeated", - "type": "DatabaseColumn", - "longType": "DatabaseColumn", - "fullType": "mgmt.v1alpha1.DatabaseColumn", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionUniqueConstraintsRequest", - "longName": "GetConnectionUniqueConstraintsRequest", - "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest", - "description": "Unique constraints for a specific table", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetConnectionUniqueConstraintsResponse", - "longName": "GetConnectionUniqueConstraintsResponse", - "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table_constraints", - "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the unique constraint", - "label": "repeated", - "type": "TableConstraintsEntry", - "longType": "GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", - "fullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", - "ismap": true, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TableConstraintsEntry", - "longName": "GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", - "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "key", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "value", - "description": "", - "label": "", - "type": "UniqueConstraint", - "longType": "UniqueConstraint", - "fullType": "mgmt.v1alpha1.UniqueConstraint", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "InitStatementOptions", - "longName": "InitStatementOptions", - "fullName": "mgmt.v1alpha1.InitStatementOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "init_schema", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "truncate_before_insert", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "truncate_cascade", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlSchemaConfig", - "longName": "MysqlSchemaConfig", - "fullName": "mgmt.v1alpha1.MysqlSchemaConfig", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "MysqlStreamConfig", - "longName": "MysqlStreamConfig", - "fullName": "mgmt.v1alpha1.MysqlStreamConfig", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "PostgresSchemaConfig", - "longName": "PostgresSchemaConfig", - "fullName": "mgmt.v1alpha1.PostgresSchemaConfig", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "PostgresStreamConfig", - "longName": "PostgresStreamConfig", - "fullName": "mgmt.v1alpha1.PostgresStreamConfig", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "PrimaryConstraint", - "longName": "PrimaryConstraint", - "fullName": "mgmt.v1alpha1.PrimaryConstraint", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "columns", - "description": "", - "label": "repeated", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UniqueConstraint", - "longName": "UniqueConstraint", - "fullName": "mgmt.v1alpha1.UniqueConstraint", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "columns", - "description": "", - "label": "repeated", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "ConnectionDataService", - "longName": "ConnectionDataService", - "fullName": "mgmt.v1alpha1.ConnectionDataService", - "description": "Service for managing connection data.\nThis is used in handle data from a connection", - "methods": [ - { - "name": "GetConnectionDataStream", - "description": "Streaming endpoint that will stream the data available from the Connection to the client.\nUsed primarily by the CLI sync command.", - "requestType": "GetConnectionDataStreamRequest", - "requestLongType": "GetConnectionDataStreamRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionDataStreamRequest", - "requestStreaming": false, - "responseType": "GetConnectionDataStreamResponse", - "responseLongType": "GetConnectionDataStreamResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionDataStreamResponse", - "responseStreaming": true - }, - { - "name": "GetConnectionSchema", - "description": "Returns the schema for a specific connection. Used mostly for SQL-based connections", - "requestType": "GetConnectionSchemaRequest", - "requestLongType": "GetConnectionSchemaRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionSchemaRequest", - "requestStreaming": false, - "responseType": "GetConnectionSchemaResponse", - "responseLongType": "GetConnectionSchemaResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionSchemaResponse", - "responseStreaming": false - }, - { - "name": "GetConnectionForeignConstraints", - "description": "For a specific connection, returns the foreign key constraints. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to determine stream order.", - "requestType": "GetConnectionForeignConstraintsRequest", - "requestLongType": "GetConnectionForeignConstraintsRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest", - "requestStreaming": false, - "responseType": "GetConnectionForeignConstraintsResponse", - "responseLongType": "GetConnectionForeignConstraintsResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse", - "responseStreaming": false - }, - { - "name": "GetConnectionPrimaryConstraints", - "description": "For a specific connection, returns the primary key constraints. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to determine stream order.", - "requestType": "GetConnectionPrimaryConstraintsRequest", - "requestLongType": "GetConnectionPrimaryConstraintsRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest", - "requestStreaming": false, - "responseType": "GetConnectionPrimaryConstraintsResponse", - "responseLongType": "GetConnectionPrimaryConstraintsResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse", - "responseStreaming": false - }, - { - "name": "GetConnectionInitStatements", - "description": "For a specific connection, returns the init table statements. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to create table schema init statement.", - "requestType": "GetConnectionInitStatementsRequest", - "requestLongType": "GetConnectionInitStatementsRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionInitStatementsRequest", - "requestStreaming": false, - "responseType": "GetConnectionInitStatementsResponse", - "responseLongType": "GetConnectionInitStatementsResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse", - "responseStreaming": false - }, - { - "name": "GetConnectionUniqueConstraints", - "description": "For a specific connection, returns the unique constraints. Mostly useful for SQL-based connections.", - "requestType": "GetConnectionUniqueConstraintsRequest", - "requestLongType": "GetConnectionUniqueConstraintsRequest", - "requestFullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest", - "requestStreaming": false, - "responseType": "GetConnectionUniqueConstraintsResponse", - "responseLongType": "GetConnectionUniqueConstraintsResponse", - "responseFullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/transformer.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": true, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [ - { - "name": "SupportedJobType", - "longName": "SupportedJobType", - "fullName": "mgmt.v1alpha1.SupportedJobType", - "description": "", - "values": [ - { - "name": "SUPPORTED_JOB_TYPE_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "SUPPORTED_JOB_TYPE_SYNC", - "number": "1", - "description": "" - }, - { - "name": "SUPPORTED_JOB_TYPE_GENERATE", - "number": "2", - "description": "" - } - ] - }, - { - "name": "TransformerDataType", - "longName": "TransformerDataType", - "fullName": "mgmt.v1alpha1.TransformerDataType", - "description": "", - "values": [ - { - "name": "TRANSFORMER_DATA_TYPE_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_STRING", - "number": "1", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_INT64", - "number": "2", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_BOOLEAN", - "number": "3", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_FLOAT64", - "number": "4", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_NULL", - "number": "5", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_ANY", - "number": "6", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_TIME", - "number": "7", - "description": "" - }, - { - "name": "TRANSFORMER_DATA_TYPE_UUID", - "number": "8", - "description": "" - } - ] - }, - { - "name": "TransformerSource", - "longName": "TransformerSource", - "fullName": "mgmt.v1alpha1.TransformerSource", - "description": "", - "values": [ - { - "name": "TRANSFORMER_SOURCE_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_PASSTHROUGH", - "number": "1", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_DEFAULT", - "number": "2", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT", - "number": "3", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_EMAIL", - "number": "4", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_EMAIL", - "number": "5", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_BOOL", - "number": "6", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER", - "number": "7", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_CITY", - "number": "8", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER", - "number": "9", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_FIRST_NAME", - "number": "10", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_FLOAT64", - "number": "11", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS", - "number": "12", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_FULL_NAME", - "number": "13", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_GENDER", - "number": "14", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER", - "number": "15", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_INT64", - "number": "16", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64", - "number": "17", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_LAST_NAME", - "number": "18", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_SHA256HASH", - "number": "19", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_SSN", - "number": "20", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_STATE", - "number": "21", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS", - "number": "22", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER", - "number": "23", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_STRING", - "number": "24", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING", - "number": "25", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP", - "number": "26", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_USERNAME", - "number": "27", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP", - "number": "28", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_UUID", - "number": "29", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_ZIPCODE", - "number": "30", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER", - "number": "31", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME", - "number": "32", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_FLOAT64", - "number": "33", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME", - "number": "34", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER", - "number": "35", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_INT64", - "number": "36", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME", - "number": "37", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER", - "number": "38", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_STRING", - "number": "39", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_NULL", - "number": "40", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_CATEGORICAL", - "number": "42", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE", - "number": "43", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_USER_DEFINED", - "number": "44", - "description": "" - }, - { - "name": "TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT", - "number": "45", - "description": "" - } - ] - } - ], - "extensions": [], - "messages": [ - { - "name": "CreateUserDefinedTransformerRequest", - "longName": "CreateUserDefinedTransformerRequest", - "fullName": "mgmt.v1alpha1.CreateUserDefinedTransformerRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "description", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "type", - "description": "@deprecated", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "TransformerSource", - "longType": "TransformerSource", - "fullType": "mgmt.v1alpha1.TransformerSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "transformer_config", - "description": "", - "label": "", - "type": "TransformerConfig", - "longType": "TransformerConfig", - "fullType": "mgmt.v1alpha1.TransformerConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateUserDefinedTransformerResponse", - "longName": "CreateUserDefinedTransformerResponse", - "fullName": "mgmt.v1alpha1.CreateUserDefinedTransformerResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer", - "description": "", - "label": "", - "type": "UserDefinedTransformer", - "longType": "UserDefinedTransformer", - "fullType": "mgmt.v1alpha1.UserDefinedTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteUserDefinedTransformerRequest", - "longName": "DeleteUserDefinedTransformerRequest", - "fullName": "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteUserDefinedTransformerResponse", - "longName": "DeleteUserDefinedTransformerResponse", - "fullName": "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateBool", - "longName": "GenerateBool", - "fullName": "mgmt.v1alpha1.GenerateBool", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateCardNumber", - "longName": "GenerateCardNumber", - "fullName": "mgmt.v1alpha1.GenerateCardNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "valid_luhn", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateCategorical", - "longName": "GenerateCategorical", - "fullName": "mgmt.v1alpha1.GenerateCategorical", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "categories", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateCity", - "longName": "GenerateCity", - "fullName": "mgmt.v1alpha1.GenerateCity", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateDefault", - "longName": "GenerateDefault", - "fullName": "mgmt.v1alpha1.GenerateDefault", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateE164PhoneNumber", - "longName": "GenerateE164PhoneNumber", - "fullName": "mgmt.v1alpha1.GenerateE164PhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "min", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateEmail", - "longName": "GenerateEmail", - "fullName": "mgmt.v1alpha1.GenerateEmail", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateFirstName", - "longName": "GenerateFirstName", - "fullName": "mgmt.v1alpha1.GenerateFirstName", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateFloat64", - "longName": "GenerateFloat64", - "fullName": "mgmt.v1alpha1.GenerateFloat64", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "randomize_sign", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "min", - "description": "", - "label": "", - "type": "double", - "longType": "double", - "fullType": "double", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max", - "description": "", - "label": "", - "type": "double", - "longType": "double", - "fullType": "double", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "precision", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateFullAddress", - "longName": "GenerateFullAddress", - "fullName": "mgmt.v1alpha1.GenerateFullAddress", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateFullName", - "longName": "GenerateFullName", - "fullName": "mgmt.v1alpha1.GenerateFullName", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateGender", - "longName": "GenerateGender", - "fullName": "mgmt.v1alpha1.GenerateGender", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "abbreviate", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateInt64", - "longName": "GenerateInt64", - "fullName": "mgmt.v1alpha1.GenerateInt64", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "randomize_sign", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "min", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateInt64PhoneNumber", - "longName": "GenerateInt64PhoneNumber", - "fullName": "mgmt.v1alpha1.GenerateInt64PhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateJavascript", - "longName": "GenerateJavascript", - "fullName": "mgmt.v1alpha1.GenerateJavascript", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "code", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateLastName", - "longName": "GenerateLastName", - "fullName": "mgmt.v1alpha1.GenerateLastName", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateSSN", - "longName": "GenerateSSN", - "fullName": "mgmt.v1alpha1.GenerateSSN", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateSha256Hash", - "longName": "GenerateSha256Hash", - "fullName": "mgmt.v1alpha1.GenerateSha256Hash", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateState", - "longName": "GenerateState", - "fullName": "mgmt.v1alpha1.GenerateState", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateStreetAddress", - "longName": "GenerateStreetAddress", - "fullName": "mgmt.v1alpha1.GenerateStreetAddress", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateString", - "longName": "GenerateString", - "fullName": "mgmt.v1alpha1.GenerateString", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "min", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateStringPhoneNumber", - "longName": "GenerateStringPhoneNumber", - "fullName": "mgmt.v1alpha1.GenerateStringPhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "min", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateUnixTimestamp", - "longName": "GenerateUnixTimestamp", - "fullName": "mgmt.v1alpha1.GenerateUnixTimestamp", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateUsername", - "longName": "GenerateUsername", - "fullName": "mgmt.v1alpha1.GenerateUsername", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateUtcTimestamp", - "longName": "GenerateUtcTimestamp", - "fullName": "mgmt.v1alpha1.GenerateUtcTimestamp", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateUuid", - "longName": "GenerateUuid", - "fullName": "mgmt.v1alpha1.GenerateUuid", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "include_hyphens", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateZipcode", - "longName": "GenerateZipcode", - "fullName": "mgmt.v1alpha1.GenerateZipcode", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetSystemTransformerBySourceRequest", - "longName": "GetSystemTransformerBySourceRequest", - "fullName": "mgmt.v1alpha1.GetSystemTransformerBySourceRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "source", - "description": "", - "label": "", - "type": "TransformerSource", - "longType": "TransformerSource", - "fullType": "mgmt.v1alpha1.TransformerSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetSystemTransformerBySourceResponse", - "longName": "GetSystemTransformerBySourceResponse", - "fullName": "mgmt.v1alpha1.GetSystemTransformerBySourceResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer", - "description": "", - "label": "", - "type": "SystemTransformer", - "longType": "SystemTransformer", - "fullType": "mgmt.v1alpha1.SystemTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetSystemTransformersRequest", - "longName": "GetSystemTransformersRequest", - "fullName": "mgmt.v1alpha1.GetSystemTransformersRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetSystemTransformersResponse", - "longName": "GetSystemTransformersResponse", - "fullName": "mgmt.v1alpha1.GetSystemTransformersResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformers", - "description": "", - "label": "repeated", - "type": "SystemTransformer", - "longType": "SystemTransformer", - "fullType": "mgmt.v1alpha1.SystemTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserDefinedTransformerByIdRequest", - "longName": "GetUserDefinedTransformerByIdRequest", - "fullName": "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserDefinedTransformerByIdResponse", - "longName": "GetUserDefinedTransformerByIdResponse", - "fullName": "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer", - "description": "", - "label": "", - "type": "UserDefinedTransformer", - "longType": "UserDefinedTransformer", - "fullType": "mgmt.v1alpha1.UserDefinedTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserDefinedTransformersRequest", - "longName": "GetUserDefinedTransformersRequest", - "fullName": "mgmt.v1alpha1.GetUserDefinedTransformersRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserDefinedTransformersResponse", - "longName": "GetUserDefinedTransformersResponse", - "fullName": "mgmt.v1alpha1.GetUserDefinedTransformersResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformers", - "description": "", - "label": "repeated", - "type": "UserDefinedTransformer", - "longType": "UserDefinedTransformer", - "fullType": "mgmt.v1alpha1.UserDefinedTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsTransformerNameAvailableRequest", - "longName": "IsTransformerNameAvailableRequest", - "fullName": "mgmt.v1alpha1.IsTransformerNameAvailableRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "transformer_name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsTransformerNameAvailableResponse", - "longName": "IsTransformerNameAvailableResponse", - "fullName": "mgmt.v1alpha1.IsTransformerNameAvailableResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "is_available", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "Null", - "longName": "Null", - "fullName": "mgmt.v1alpha1.Null", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "Passthrough", - "longName": "Passthrough", - "fullName": "mgmt.v1alpha1.Passthrough", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "SystemTransformer", - "longName": "SystemTransformer", - "fullName": "mgmt.v1alpha1.SystemTransformer", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "description", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "data_type", - "description": "", - "label": "", - "type": "TransformerDataType", - "longType": "TransformerDataType", - "fullType": "mgmt.v1alpha1.TransformerDataType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "TransformerSource", - "longType": "TransformerSource", - "fullType": "mgmt.v1alpha1.TransformerSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "config", - "description": "", - "label": "", - "type": "TransformerConfig", - "longType": "TransformerConfig", - "fullType": "mgmt.v1alpha1.TransformerConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "data_types", - "description": "", - "label": "repeated", - "type": "TransformerDataType", - "longType": "TransformerDataType", - "fullType": "mgmt.v1alpha1.TransformerDataType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "supported_job_types", - "description": "", - "label": "repeated", - "type": "SupportedJobType", - "longType": "SupportedJobType", - "fullType": "mgmt.v1alpha1.SupportedJobType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformCharacterScramble", - "longName": "TransformCharacterScramble", - "fullName": "mgmt.v1alpha1.TransformCharacterScramble", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "user_provided_regex", - "description": "a user provided regular expression that they wish to validate if it compiles in GO", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_user_provided_regex", - "defaultValue": "" - } - ] - }, - { - "name": "TransformE164PhoneNumber", - "longName": "TransformE164PhoneNumber", - "fullName": "mgmt.v1alpha1.TransformE164PhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformEmail", - "longName": "TransformEmail", - "fullName": "mgmt.v1alpha1.TransformEmail", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_domain", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "excluded_domains", - "description": "", - "label": "repeated", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformFirstName", - "longName": "TransformFirstName", - "fullName": "mgmt.v1alpha1.TransformFirstName", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformFloat64", - "longName": "TransformFloat64", - "fullName": "mgmt.v1alpha1.TransformFloat64", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "randomization_range_min", - "description": "", - "label": "", - "type": "double", - "longType": "double", - "fullType": "double", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "randomization_range_max", - "description": "", - "label": "", - "type": "double", - "longType": "double", - "fullType": "double", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformFullName", - "longName": "TransformFullName", - "fullName": "mgmt.v1alpha1.TransformFullName", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformInt64", - "longName": "TransformInt64", - "fullName": "mgmt.v1alpha1.TransformInt64", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "randomization_range_min", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "randomization_range_max", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformInt64PhoneNumber", - "longName": "TransformInt64PhoneNumber", - "fullName": "mgmt.v1alpha1.TransformInt64PhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformJavascript", - "longName": "TransformJavascript", - "fullName": "mgmt.v1alpha1.TransformJavascript", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "code", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformLastName", - "longName": "TransformLastName", - "fullName": "mgmt.v1alpha1.TransformLastName", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformPhoneNumber", - "longName": "TransformPhoneNumber", - "fullName": "mgmt.v1alpha1.TransformPhoneNumber", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformString", - "longName": "TransformString", - "fullName": "mgmt.v1alpha1.TransformString", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "preserve_length", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TransformerConfig", - "longName": "TransformerConfig", - "fullName": "mgmt.v1alpha1.TransformerConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "generate_email_config", - "description": "", - "label": "", - "type": "GenerateEmail", - "longType": "GenerateEmail", - "fullType": "mgmt.v1alpha1.GenerateEmail", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_email_config", - "description": "", - "label": "", - "type": "TransformEmail", - "longType": "TransformEmail", - "fullType": "mgmt.v1alpha1.TransformEmail", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_bool_config", - "description": "", - "label": "", - "type": "GenerateBool", - "longType": "GenerateBool", - "fullType": "mgmt.v1alpha1.GenerateBool", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_card_number_config", - "description": "", - "label": "", - "type": "GenerateCardNumber", - "longType": "GenerateCardNumber", - "fullType": "mgmt.v1alpha1.GenerateCardNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_city_config", - "description": "", - "label": "", - "type": "GenerateCity", - "longType": "GenerateCity", - "fullType": "mgmt.v1alpha1.GenerateCity", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_e164_phone_number_config", - "description": "", - "label": "", - "type": "GenerateE164PhoneNumber", - "longType": "GenerateE164PhoneNumber", - "fullType": "mgmt.v1alpha1.GenerateE164PhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_first_name_config", - "description": "", - "label": "", - "type": "GenerateFirstName", - "longType": "GenerateFirstName", - "fullType": "mgmt.v1alpha1.GenerateFirstName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_float64_config", - "description": "", - "label": "", - "type": "GenerateFloat64", - "longType": "GenerateFloat64", - "fullType": "mgmt.v1alpha1.GenerateFloat64", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_full_address_config", - "description": "", - "label": "", - "type": "GenerateFullAddress", - "longType": "GenerateFullAddress", - "fullType": "mgmt.v1alpha1.GenerateFullAddress", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_full_name_config", - "description": "", - "label": "", - "type": "GenerateFullName", - "longType": "GenerateFullName", - "fullType": "mgmt.v1alpha1.GenerateFullName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_gender_config", - "description": "", - "label": "", - "type": "GenerateGender", - "longType": "GenerateGender", - "fullType": "mgmt.v1alpha1.GenerateGender", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_int64_phone_number_config", - "description": "", - "label": "", - "type": "GenerateInt64PhoneNumber", - "longType": "GenerateInt64PhoneNumber", - "fullType": "mgmt.v1alpha1.GenerateInt64PhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_int64_config", - "description": "", - "label": "", - "type": "GenerateInt64", - "longType": "GenerateInt64", - "fullType": "mgmt.v1alpha1.GenerateInt64", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_last_name_config", - "description": "", - "label": "", - "type": "GenerateLastName", - "longType": "GenerateLastName", - "fullType": "mgmt.v1alpha1.GenerateLastName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_sha256hash_config", - "description": "", - "label": "", - "type": "GenerateSha256Hash", - "longType": "GenerateSha256Hash", - "fullType": "mgmt.v1alpha1.GenerateSha256Hash", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_ssn_config", - "description": "", - "label": "", - "type": "GenerateSSN", - "longType": "GenerateSSN", - "fullType": "mgmt.v1alpha1.GenerateSSN", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_state_config", - "description": "", - "label": "", - "type": "GenerateState", - "longType": "GenerateState", - "fullType": "mgmt.v1alpha1.GenerateState", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_street_address_config", - "description": "", - "label": "", - "type": "GenerateStreetAddress", - "longType": "GenerateStreetAddress", - "fullType": "mgmt.v1alpha1.GenerateStreetAddress", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_string_phone_number_config", - "description": "", - "label": "", - "type": "GenerateStringPhoneNumber", - "longType": "GenerateStringPhoneNumber", - "fullType": "mgmt.v1alpha1.GenerateStringPhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_string_config", - "description": "", - "label": "", - "type": "GenerateString", - "longType": "GenerateString", - "fullType": "mgmt.v1alpha1.GenerateString", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_unixtimestamp_config", - "description": "", - "label": "", - "type": "GenerateUnixTimestamp", - "longType": "GenerateUnixTimestamp", - "fullType": "mgmt.v1alpha1.GenerateUnixTimestamp", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_username_config", - "description": "", - "label": "", - "type": "GenerateUsername", - "longType": "GenerateUsername", - "fullType": "mgmt.v1alpha1.GenerateUsername", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_utctimestamp_config", - "description": "", - "label": "", - "type": "GenerateUtcTimestamp", - "longType": "GenerateUtcTimestamp", - "fullType": "mgmt.v1alpha1.GenerateUtcTimestamp", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_uuid_config", - "description": "", - "label": "", - "type": "GenerateUuid", - "longType": "GenerateUuid", - "fullType": "mgmt.v1alpha1.GenerateUuid", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_zipcode_config", - "description": "", - "label": "", - "type": "GenerateZipcode", - "longType": "GenerateZipcode", - "fullType": "mgmt.v1alpha1.GenerateZipcode", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_e164_phone_number_config", - "description": "", - "label": "", - "type": "TransformE164PhoneNumber", - "longType": "TransformE164PhoneNumber", - "fullType": "mgmt.v1alpha1.TransformE164PhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_first_name_config", - "description": "", - "label": "", - "type": "TransformFirstName", - "longType": "TransformFirstName", - "fullType": "mgmt.v1alpha1.TransformFirstName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_float64_config", - "description": "", - "label": "", - "type": "TransformFloat64", - "longType": "TransformFloat64", - "fullType": "mgmt.v1alpha1.TransformFloat64", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_full_name_config", - "description": "", - "label": "", - "type": "TransformFullName", - "longType": "TransformFullName", - "fullType": "mgmt.v1alpha1.TransformFullName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_int64_phone_number_config", - "description": "", - "label": "", - "type": "TransformInt64PhoneNumber", - "longType": "TransformInt64PhoneNumber", - "fullType": "mgmt.v1alpha1.TransformInt64PhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_int64_config", - "description": "", - "label": "", - "type": "TransformInt64", - "longType": "TransformInt64", - "fullType": "mgmt.v1alpha1.TransformInt64", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_last_name_config", - "description": "", - "label": "", - "type": "TransformLastName", - "longType": "TransformLastName", - "fullType": "mgmt.v1alpha1.TransformLastName", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_phone_number_config", - "description": "", - "label": "", - "type": "TransformPhoneNumber", - "longType": "TransformPhoneNumber", - "fullType": "mgmt.v1alpha1.TransformPhoneNumber", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_string_config", - "description": "", - "label": "", - "type": "TransformString", - "longType": "TransformString", - "fullType": "mgmt.v1alpha1.TransformString", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "passthrough_config", - "description": "", - "label": "", - "type": "Passthrough", - "longType": "Passthrough", - "fullType": "mgmt.v1alpha1.Passthrough", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "nullconfig", - "description": "", - "label": "", - "type": "Null", - "longType": "Null", - "fullType": "mgmt.v1alpha1.Null", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "user_defined_transformer_config", - "description": "", - "label": "", - "type": "UserDefinedTransformerConfig", - "longType": "UserDefinedTransformerConfig", - "fullType": "mgmt.v1alpha1.UserDefinedTransformerConfig", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_default_config", - "description": "", - "label": "", - "type": "GenerateDefault", - "longType": "GenerateDefault", - "fullType": "mgmt.v1alpha1.GenerateDefault", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_javascript_config", - "description": "", - "label": "", - "type": "TransformJavascript", - "longType": "TransformJavascript", - "fullType": "mgmt.v1alpha1.TransformJavascript", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_categorical_config", - "description": "", - "label": "", - "type": "GenerateCategorical", - "longType": "GenerateCategorical", - "fullType": "mgmt.v1alpha1.GenerateCategorical", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "transform_character_scramble_config", - "description": "", - "label": "", - "type": "TransformCharacterScramble", - "longType": "TransformCharacterScramble", - "fullType": "mgmt.v1alpha1.TransformCharacterScramble", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate_javascript_config", - "description": "", - "label": "", - "type": "GenerateJavascript", - "longType": "GenerateJavascript", - "fullType": "mgmt.v1alpha1.GenerateJavascript", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateUserDefinedTransformerRequest", - "longName": "UpdateUserDefinedTransformerRequest", - "fullName": "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "description", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "transformer_config", - "description": "", - "label": "", - "type": "TransformerConfig", - "longType": "TransformerConfig", - "fullType": "mgmt.v1alpha1.TransformerConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateUserDefinedTransformerResponse", - "longName": "UpdateUserDefinedTransformerResponse", - "fullName": "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "transformer", - "description": "", - "label": "", - "type": "UserDefinedTransformer", - "longType": "UserDefinedTransformer", - "fullType": "mgmt.v1alpha1.UserDefinedTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UserDefinedTransformer", - "longName": "UserDefinedTransformer", - "fullName": "mgmt.v1alpha1.UserDefinedTransformer", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "description", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "data_type", - "description": "This property is readonly and is calculated based off the origin system transformer", - "label": "", - "type": "TransformerDataType", - "longType": "TransformerDataType", - "fullType": "mgmt.v1alpha1.TransformerDataType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "TransformerSource", - "longType": "TransformerSource", - "fullType": "mgmt.v1alpha1.TransformerSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "config", - "description": "", - "label": "", - "type": "TransformerConfig", - "longType": "TransformerConfig", - "fullType": "mgmt.v1alpha1.TransformerConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "data_types", - "description": "", - "label": "repeated", - "type": "TransformerDataType", - "longType": "TransformerDataType", - "fullType": "mgmt.v1alpha1.TransformerDataType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UserDefinedTransformerConfig", - "longName": "UserDefinedTransformerConfig", - "fullName": "mgmt.v1alpha1.UserDefinedTransformerConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ValidateUserJavascriptCodeRequest", - "longName": "ValidateUserJavascriptCodeRequest", - "fullName": "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "code", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ValidateUserJavascriptCodeResponse", - "longName": "ValidateUserJavascriptCodeResponse", - "fullName": "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "valid", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ValidateUserRegexCodeRequest", - "longName": "ValidateUserRegexCodeRequest", - "fullName": "mgmt.v1alpha1.ValidateUserRegexCodeRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "user_provided_regex", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ValidateUserRegexCodeResponse", - "longName": "ValidateUserRegexCodeResponse", - "fullName": "mgmt.v1alpha1.ValidateUserRegexCodeResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "valid", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "TransformersService", - "longName": "TransformersService", - "fullName": "mgmt.v1alpha1.TransformersService", - "description": "", - "methods": [ - { - "name": "GetSystemTransformers", - "description": "", - "requestType": "GetSystemTransformersRequest", - "requestLongType": "GetSystemTransformersRequest", - "requestFullType": "mgmt.v1alpha1.GetSystemTransformersRequest", - "requestStreaming": false, - "responseType": "GetSystemTransformersResponse", - "responseLongType": "GetSystemTransformersResponse", - "responseFullType": "mgmt.v1alpha1.GetSystemTransformersResponse", - "responseStreaming": false - }, - { - "name": "GetSystemTransformerBySource", - "description": "", - "requestType": "GetSystemTransformerBySourceRequest", - "requestLongType": "GetSystemTransformerBySourceRequest", - "requestFullType": "mgmt.v1alpha1.GetSystemTransformerBySourceRequest", - "requestStreaming": false, - "responseType": "GetSystemTransformerBySourceResponse", - "responseLongType": "GetSystemTransformerBySourceResponse", - "responseFullType": "mgmt.v1alpha1.GetSystemTransformerBySourceResponse", - "responseStreaming": false - }, - { - "name": "GetUserDefinedTransformers", - "description": "", - "requestType": "GetUserDefinedTransformersRequest", - "requestLongType": "GetUserDefinedTransformersRequest", - "requestFullType": "mgmt.v1alpha1.GetUserDefinedTransformersRequest", - "requestStreaming": false, - "responseType": "GetUserDefinedTransformersResponse", - "responseLongType": "GetUserDefinedTransformersResponse", - "responseFullType": "mgmt.v1alpha1.GetUserDefinedTransformersResponse", - "responseStreaming": false - }, - { - "name": "GetUserDefinedTransformerById", - "description": "", - "requestType": "GetUserDefinedTransformerByIdRequest", - "requestLongType": "GetUserDefinedTransformerByIdRequest", - "requestFullType": "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest", - "requestStreaming": false, - "responseType": "GetUserDefinedTransformerByIdResponse", - "responseLongType": "GetUserDefinedTransformerByIdResponse", - "responseFullType": "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse", - "responseStreaming": false - }, - { - "name": "CreateUserDefinedTransformer", - "description": "", - "requestType": "CreateUserDefinedTransformerRequest", - "requestLongType": "CreateUserDefinedTransformerRequest", - "requestFullType": "mgmt.v1alpha1.CreateUserDefinedTransformerRequest", - "requestStreaming": false, - "responseType": "CreateUserDefinedTransformerResponse", - "responseLongType": "CreateUserDefinedTransformerResponse", - "responseFullType": "mgmt.v1alpha1.CreateUserDefinedTransformerResponse", - "responseStreaming": false - }, - { - "name": "DeleteUserDefinedTransformer", - "description": "", - "requestType": "DeleteUserDefinedTransformerRequest", - "requestLongType": "DeleteUserDefinedTransformerRequest", - "requestFullType": "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest", - "requestStreaming": false, - "responseType": "DeleteUserDefinedTransformerResponse", - "responseLongType": "DeleteUserDefinedTransformerResponse", - "responseFullType": "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse", - "responseStreaming": false - }, - { - "name": "UpdateUserDefinedTransformer", - "description": "", - "requestType": "UpdateUserDefinedTransformerRequest", - "requestLongType": "UpdateUserDefinedTransformerRequest", - "requestFullType": "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest", - "requestStreaming": false, - "responseType": "UpdateUserDefinedTransformerResponse", - "responseLongType": "UpdateUserDefinedTransformerResponse", - "responseFullType": "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse", - "responseStreaming": false - }, - { - "name": "IsTransformerNameAvailable", - "description": "", - "requestType": "IsTransformerNameAvailableRequest", - "requestLongType": "IsTransformerNameAvailableRequest", - "requestFullType": "mgmt.v1alpha1.IsTransformerNameAvailableRequest", - "requestStreaming": false, - "responseType": "IsTransformerNameAvailableResponse", - "responseLongType": "IsTransformerNameAvailableResponse", - "responseFullType": "mgmt.v1alpha1.IsTransformerNameAvailableResponse", - "responseStreaming": false - }, - { - "name": "ValidateUserJavascriptCode", - "description": "", - "requestType": "ValidateUserJavascriptCodeRequest", - "requestLongType": "ValidateUserJavascriptCodeRequest", - "requestFullType": "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest", - "requestStreaming": false, - "responseType": "ValidateUserJavascriptCodeResponse", - "responseLongType": "ValidateUserJavascriptCodeResponse", - "responseFullType": "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse", - "responseStreaming": false - }, - { - "name": "ValidateUserRegexCode", - "description": "", - "requestType": "ValidateUserRegexCodeRequest", - "requestLongType": "ValidateUserRegexCodeRequest", - "requestFullType": "mgmt.v1alpha1.ValidateUserRegexCodeRequest", - "requestStreaming": false, - "responseType": "ValidateUserRegexCodeResponse", - "responseLongType": "ValidateUserRegexCodeResponse", - "responseFullType": "mgmt.v1alpha1.ValidateUserRegexCodeResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/job.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": true, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [ - { - "name": "ActivityStatus", - "longName": "ActivityStatus", - "fullName": "mgmt.v1alpha1.ActivityStatus", - "description": "", - "values": [ - { - "name": "ACTIVITY_STATUS_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "ACTIVITY_STATUS_SCHEDULED", - "number": "1", - "description": "" - }, - { - "name": "ACTIVITY_STATUS_STARTED", - "number": "2", - "description": "" - }, - { - "name": "ACTIVITY_STATUS_CANCELED", - "number": "3", - "description": "" - }, - { - "name": "ACTIVITY_STATUS_FAILED", - "number": "4", - "description": "" - } - ] - }, - { - "name": "JobRunStatus", - "longName": "JobRunStatus", - "fullName": "mgmt.v1alpha1.JobRunStatus", - "description": "An enumeration of job run statuses.", - "values": [ - { - "name": "JOB_RUN_STATUS_UNSPECIFIED", - "number": "0", - "description": "if the job run status is unknown" - }, - { - "name": "JOB_RUN_STATUS_PENDING", - "number": "1", - "description": "the run is pending and has not started yet" - }, - { - "name": "JOB_RUN_STATUS_RUNNING", - "number": "2", - "description": "the run is currently in progress" - }, - { - "name": "JOB_RUN_STATUS_COMPLETE", - "number": "3", - "description": "the run has successfully completed" - }, - { - "name": "JOB_RUN_STATUS_ERROR", - "number": "4", - "description": "the run ended with an error" - }, - { - "name": "JOB_RUN_STATUS_CANCELED", - "number": "5", - "description": "the run was cancelled" - }, - { - "name": "JOB_RUN_STATUS_TERMINATED", - "number": "6", - "description": "the run was terminated" - }, - { - "name": "JOB_RUN_STATUS_FAILED", - "number": "7", - "description": "the run ended in failure" - }, - { - "name": "JOB_RUN_STATUS_TIMED_OUT", - "number": "8", - "description": "the run was ended pre-maturely due to timeout" - } - ] - }, - { - "name": "JobStatus", - "longName": "JobStatus", - "fullName": "mgmt.v1alpha1.JobStatus", - "description": "", - "values": [ - { - "name": "JOB_STATUS_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "JOB_STATUS_ENABLED", - "number": "1", - "description": "" - }, - { - "name": "JOB_STATUS_PAUSED", - "number": "3", - "description": "" - }, - { - "name": "JOB_STATUS_DISABLED", - "number": "4", - "description": "" - } - ] - }, - { - "name": "LogLevel", - "longName": "LogLevel", - "fullName": "mgmt.v1alpha1.LogLevel", - "description": "", - "values": [ - { - "name": "LOG_LEVEL_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "LOG_LEVEL_DEBUG", - "number": "1", - "description": "" - }, - { - "name": "LOG_LEVEL_INFO", - "number": "2", - "description": "" - }, - { - "name": "LOG_LEVEL_WARN", - "number": "3", - "description": "" - }, - { - "name": "LOG_LEVEL_ERROR", - "number": "4", - "description": "" - } - ] - }, - { - "name": "LogWindow", - "longName": "LogWindow", - "fullName": "mgmt.v1alpha1.LogWindow", - "description": "", - "values": [ - { - "name": "LOG_WINDOW_NO_TIME_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "LOG_WINDOW_FIFTEEN_MIN", - "number": "1", - "description": "" - }, - { - "name": "LOG_WINDOW_ONE_HOUR", - "number": "2", - "description": "" - }, - { - "name": "LOG_WINDOW_ONE_DAY", - "number": "3", - "description": "" - } - ] - } - ], - "extensions": [], - "messages": [ - { - "name": "ActivityFailure", - "longName": "ActivityFailure", - "fullName": "mgmt.v1alpha1.ActivityFailure", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "message", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ActivityOptions", - "longName": "ActivityOptions", - "fullName": "mgmt.v1alpha1.ActivityOptions", - "description": "Config that contains various timeouts that are configured in the underlying temporal workflow(s) and activities", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "schedule_to_close_timeout", - "description": "Total time that a workflow is willing to wait for an activity to complete, including retries.\nMeasured in seconds", - "label": "optional", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": true, - "oneofdecl": "_schedule_to_close_timeout", - "defaultValue": "" - }, - { - "name": "start_to_close_timeout", - "description": "Max time of a single Temporal Activity execution attempt.\nThis timeout should be as short as the longest psosible execution of any activity (e.g. table sync).\nImportant to know that this is per retry attempt. Defaults to the schedule to close timeout if not provided.\nMeasured in seconds", - "label": "optional", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": true, - "oneofdecl": "_start_to_close_timeout", - "defaultValue": "" - }, - { - "name": "retry_policy", - "description": "Optionally define a retry policy for the activity\nIf max attempts is not set, the activity will retry indefinitely until the start to close timeout lapses", - "label": "", - "type": "RetryPolicy", - "longType": "RetryPolicy", - "fullType": "mgmt.v1alpha1.RetryPolicy", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AwsS3DestinationConnectionOptions", - "longName": "AwsS3DestinationConnectionOptions", - "fullName": "mgmt.v1alpha1.AwsS3DestinationConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "AwsS3SourceConnectionOptions", - "longName": "AwsS3SourceConnectionOptions", - "fullName": "mgmt.v1alpha1.AwsS3SourceConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CancelJobRunRequest", - "longName": "CancelJobRunRequest", - "fullName": "mgmt.v1alpha1.CancelJobRunRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CancelJobRunResponse", - "longName": "CancelJobRunResponse", - "fullName": "mgmt.v1alpha1.CancelJobRunResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "CreateJobDestination", - "longName": "CreateJobDestination", - "fullName": "mgmt.v1alpha1.CreateJobDestination", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "options", - "description": "", - "label": "", - "type": "JobDestinationOptions", - "longType": "JobDestinationOptions", - "fullType": "mgmt.v1alpha1.JobDestinationOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobDestinationConnectionsRequest", - "longName": "CreateJobDestinationConnectionsRequest", - "fullName": "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "destinations", - "description": "", - "label": "repeated", - "type": "CreateJobDestination", - "longType": "CreateJobDestination", - "fullType": "mgmt.v1alpha1.CreateJobDestination", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobDestinationConnectionsResponse", - "longName": "CreateJobDestinationConnectionsResponse", - "fullName": "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobRequest", - "longName": "CreateJobRequest", - "fullName": "mgmt.v1alpha1.CreateJobRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "The unique account identifier that this job will be associated with", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "job_name", - "description": "The unique, friendly name of the job. This is unique per account", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "cron_schedule", - "description": "Optionally provide a cron schedule. Goes into effect if the job status is set to enabled", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_cron_schedule", - "defaultValue": "" - }, - { - "name": "mappings", - "description": "", - "label": "repeated", - "type": "JobMapping", - "longType": "JobMapping", - "fullType": "mgmt.v1alpha1.JobMapping", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "JobSource", - "longType": "JobSource", - "fullType": "mgmt.v1alpha1.JobSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "destinations", - "description": "", - "label": "repeated", - "type": "CreateJobDestination", - "longType": "CreateJobDestination", - "fullType": "mgmt.v1alpha1.CreateJobDestination", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "initiate_job_run", - "description": "Initially trigger a run of this job regardless of its status or cron schedule", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "workflow_options", - "description": "Specify timeouts and other workflow options for the underlying temporal workflow", - "label": "", - "type": "WorkflowOptions", - "longType": "WorkflowOptions", - "fullType": "mgmt.v1alpha1.WorkflowOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "sync_options", - "description": "Specify timeout and retry options for data synchronization activities\nData sync activities are any piece of work that involves actually synchronizing data from a source to a destination\nFor the data sync and generate jobs, this will be applied per table", - "label": "", - "type": "ActivityOptions", - "longType": "ActivityOptions", - "fullType": "mgmt.v1alpha1.ActivityOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobResponse", - "longName": "CreateJobResponse", - "fullName": "mgmt.v1alpha1.CreateJobResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobRunRequest", - "longName": "CreateJobRunRequest", - "fullName": "mgmt.v1alpha1.CreateJobRunRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateJobRunResponse", - "longName": "CreateJobRunResponse", - "fullName": "mgmt.v1alpha1.CreateJobRunResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "DeleteJobDestinationConnectionRequest", - "longName": "DeleteJobDestinationConnectionRequest", - "fullName": "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "destination_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteJobDestinationConnectionResponse", - "longName": "DeleteJobDestinationConnectionResponse", - "fullName": "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "DeleteJobRequest", - "longName": "DeleteJobRequest", - "fullName": "mgmt.v1alpha1.DeleteJobRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteJobResponse", - "longName": "DeleteJobResponse", - "fullName": "mgmt.v1alpha1.DeleteJobResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "DeleteJobRunRequest", - "longName": "DeleteJobRunRequest", - "fullName": "mgmt.v1alpha1.DeleteJobRunRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DeleteJobRunResponse", - "longName": "DeleteJobRunResponse", - "fullName": "mgmt.v1alpha1.DeleteJobRunResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GenerateSourceOptions", - "longName": "GenerateSourceOptions", - "fullName": "mgmt.v1alpha1.GenerateSourceOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "schemas", - "description": "", - "label": "repeated", - "type": "GenerateSourceSchemaOption", - "longType": "GenerateSourceSchemaOption", - "fullType": "mgmt.v1alpha1.GenerateSourceSchemaOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "fk_source_connection_id", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_fk_source_connection_id", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateSourceSchemaOption", - "longName": "GenerateSourceSchemaOption", - "fullName": "mgmt.v1alpha1.GenerateSourceSchemaOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "tables", - "description": "", - "label": "repeated", - "type": "GenerateSourceTableOption", - "longType": "GenerateSourceTableOption", - "fullType": "mgmt.v1alpha1.GenerateSourceTableOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GenerateSourceTableOption", - "longName": "GenerateSourceTableOption", - "fullName": "mgmt.v1alpha1.GenerateSourceTableOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "row_count", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobNextRunsRequest", - "longName": "GetJobNextRunsRequest", - "fullName": "mgmt.v1alpha1.GetJobNextRunsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobNextRunsResponse", - "longName": "GetJobNextRunsResponse", - "fullName": "mgmt.v1alpha1.GetJobNextRunsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "next_runs", - "description": "", - "label": "", - "type": "JobNextRuns", - "longType": "JobNextRuns", - "fullType": "mgmt.v1alpha1.JobNextRuns", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRecentRunsRequest", - "longName": "GetJobRecentRunsRequest", - "fullName": "mgmt.v1alpha1.GetJobRecentRunsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRecentRunsResponse", - "longName": "GetJobRecentRunsResponse", - "fullName": "mgmt.v1alpha1.GetJobRecentRunsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "recent_runs", - "description": "", - "label": "repeated", - "type": "JobRecentRun", - "longType": "JobRecentRun", - "fullType": "mgmt.v1alpha1.JobRecentRun", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRequest", - "longName": "GetJobRequest", - "fullName": "mgmt.v1alpha1.GetJobRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobResponse", - "longName": "GetJobResponse", - "fullName": "mgmt.v1alpha1.GetJobResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunEventsRequest", - "longName": "GetJobRunEventsRequest", - "fullName": "mgmt.v1alpha1.GetJobRunEventsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunEventsResponse", - "longName": "GetJobRunEventsResponse", - "fullName": "mgmt.v1alpha1.GetJobRunEventsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "events", - "description": "", - "label": "repeated", - "type": "JobRunEvent", - "longType": "JobRunEvent", - "fullType": "mgmt.v1alpha1.JobRunEvent", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "is_run_complete", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunLogsStreamRequest", - "longName": "GetJobRunLogsStreamRequest", - "fullName": "mgmt.v1alpha1.GetJobRunLogsStreamRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "window", - "description": "The time window in which to retrieve the logs", - "label": "", - "type": "LogWindow", - "longType": "LogWindow", - "fullType": "mgmt.v1alpha1.LogWindow", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "should_tail", - "description": "Whether or not to tail the stream. Note: only works with k8s-pods and is not currently supported with Loki logs", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "max_log_lines", - "description": "Optionally provide a max log limit", - "label": "optional", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": true, - "oneofdecl": "_max_log_lines", - "defaultValue": "" - }, - { - "name": "log_levels", - "description": "Provide a list of log levels to filter by. If any of these are UNSPECIFIED, all log levels are returned.", - "label": "repeated", - "type": "LogLevel", - "longType": "LogLevel", - "fullType": "mgmt.v1alpha1.LogLevel", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunLogsStreamResponse", - "longName": "GetJobRunLogsStreamResponse", - "fullName": "mgmt.v1alpha1.GetJobRunLogsStreamResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "log_line", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "timestamp", - "description": "", - "label": "optional", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": true, - "oneofdecl": "_timestamp", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunRequest", - "longName": "GetJobRunRequest", - "fullName": "mgmt.v1alpha1.GetJobRunRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunResponse", - "longName": "GetJobRunResponse", - "fullName": "mgmt.v1alpha1.GetJobRunResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run", - "description": "", - "label": "", - "type": "JobRun", - "longType": "JobRun", - "fullType": "mgmt.v1alpha1.JobRun", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunsRequest", - "longName": "GetJobRunsRequest", - "fullName": "mgmt.v1alpha1.GetJobRunsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "id", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobRunsResponse", - "longName": "GetJobRunsResponse", - "fullName": "mgmt.v1alpha1.GetJobRunsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_runs", - "description": "", - "label": "repeated", - "type": "JobRun", - "longType": "JobRun", - "fullType": "mgmt.v1alpha1.JobRun", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobStatusRequest", - "longName": "GetJobStatusRequest", - "fullName": "mgmt.v1alpha1.GetJobStatusRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobStatusResponse", - "longName": "GetJobStatusResponse", - "fullName": "mgmt.v1alpha1.GetJobStatusResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "status", - "description": "", - "label": "", - "type": "JobStatus", - "longType": "JobStatus", - "fullType": "mgmt.v1alpha1.JobStatus", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobStatusesRequest", - "longName": "GetJobStatusesRequest", - "fullName": "mgmt.v1alpha1.GetJobStatusesRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobStatusesResponse", - "longName": "GetJobStatusesResponse", - "fullName": "mgmt.v1alpha1.GetJobStatusesResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "statuses", - "description": "", - "label": "repeated", - "type": "JobStatusRecord", - "longType": "JobStatusRecord", - "fullType": "mgmt.v1alpha1.JobStatusRecord", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobsRequest", - "longName": "GetJobsRequest", - "fullName": "mgmt.v1alpha1.GetJobsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetJobsResponse", - "longName": "GetJobsResponse", - "fullName": "mgmt.v1alpha1.GetJobsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "jobs", - "description": "", - "label": "repeated", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsJobNameAvailableRequest", - "longName": "IsJobNameAvailableRequest", - "fullName": "mgmt.v1alpha1.IsJobNameAvailableRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsJobNameAvailableResponse", - "longName": "IsJobNameAvailableResponse", - "fullName": "mgmt.v1alpha1.IsJobNameAvailableResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "is_available", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "Job", - "longName": "Job", - "fullName": "mgmt.v1alpha1.Job", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "The unique identifier of the job", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_by_user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_by_user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "The unique, friendly name of the job", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "JobSource", - "longType": "JobSource", - "fullType": "mgmt.v1alpha1.JobSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "destinations", - "description": "", - "label": "repeated", - "type": "JobDestination", - "longType": "JobDestination", - "fullType": "mgmt.v1alpha1.JobDestination", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "mappings", - "description": "", - "label": "repeated", - "type": "JobMapping", - "longType": "JobMapping", - "fullType": "mgmt.v1alpha1.JobMapping", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "cron_schedule", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_cron_schedule", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "The account identifier that a job is associated with", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "sync_options", - "description": "Specify timeout and retry options for data synchronization activities\nData sync activities are any piece of work that involves actually synchronizing data from a source to a destination\nFor the data sync and generate jobs, this will be applied per table", - "label": "", - "type": "ActivityOptions", - "longType": "ActivityOptions", - "fullType": "mgmt.v1alpha1.ActivityOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "workflow_options", - "description": "Specify timeouts and other workflow options for the underlying temporal workflow", - "label": "", - "type": "WorkflowOptions", - "longType": "WorkflowOptions", - "fullType": "mgmt.v1alpha1.WorkflowOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobDestination", - "longName": "JobDestination", - "fullName": "mgmt.v1alpha1.JobDestination", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "options", - "description": "", - "label": "", - "type": "JobDestinationOptions", - "longType": "JobDestinationOptions", - "fullType": "mgmt.v1alpha1.JobDestinationOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobDestinationOptions", - "longName": "JobDestinationOptions", - "fullName": "mgmt.v1alpha1.JobDestinationOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "postgres_options", - "description": "", - "label": "", - "type": "PostgresDestinationConnectionOptions", - "longType": "PostgresDestinationConnectionOptions", - "fullType": "mgmt.v1alpha1.PostgresDestinationConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "aws_s3_options", - "description": "", - "label": "", - "type": "AwsS3DestinationConnectionOptions", - "longType": "AwsS3DestinationConnectionOptions", - "fullType": "mgmt.v1alpha1.AwsS3DestinationConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "mysql_options", - "description": "", - "label": "", - "type": "MysqlDestinationConnectionOptions", - "longType": "MysqlDestinationConnectionOptions", - "fullType": "mgmt.v1alpha1.MysqlDestinationConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "JobMapping", - "longName": "JobMapping", - "fullName": "mgmt.v1alpha1.JobMapping", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "column", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "transformer", - "description": "", - "label": "", - "type": "JobMappingTransformer", - "longType": "JobMappingTransformer", - "fullType": "mgmt.v1alpha1.JobMappingTransformer", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobMappingTransformer", - "longName": "JobMappingTransformer", - "fullName": "mgmt.v1alpha1.JobMappingTransformer", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "source", - "description": "", - "label": "", - "type": "TransformerSource", - "longType": "TransformerSource", - "fullType": "mgmt.v1alpha1.TransformerSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "config", - "description": "", - "label": "", - "type": "TransformerConfig", - "longType": "TransformerConfig", - "fullType": "mgmt.v1alpha1.TransformerConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobNextRuns", - "longName": "JobNextRuns", - "fullName": "mgmt.v1alpha1.JobNextRuns", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "next_run_times", - "description": "", - "label": "repeated", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRecentRun", - "longName": "JobRecentRun", - "fullName": "mgmt.v1alpha1.JobRecentRun", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "start_time", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRun", - "longName": "JobRun", - "fullName": "mgmt.v1alpha1.JobRun", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "The id of the job run. This will currently be equivalent to the temporal workflow id", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "job_id", - "description": "The unique identifier of the job id this run is associated with", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "The name of the job run.", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "status", - "description": "the status of the job run", - "label": "", - "type": "JobRunStatus", - "longType": "JobRunStatus", - "fullType": "mgmt.v1alpha1.JobRunStatus", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "started_at", - "description": "A timestamp of when the run started", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "completed_at", - "description": "Available if the run completed or has not yet been archived by the system", - "label": "optional", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": true, - "oneofdecl": "_completed_at", - "defaultValue": "" - }, - { - "name": "pending_activities", - "description": "Pending activities are only returned when retrieving a specific job run and will not be returned when requesting job runs in list format", - "label": "repeated", - "type": "PendingActivity", - "longType": "PendingActivity", - "fullType": "mgmt.v1alpha1.PendingActivity", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRunEvent", - "longName": "JobRunEvent", - "fullName": "mgmt.v1alpha1.JobRunEvent", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "type", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "start_time", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "close_time", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "metadata", - "description": "", - "label": "", - "type": "JobRunEventMetadata", - "longType": "JobRunEventMetadata", - "fullType": "mgmt.v1alpha1.JobRunEventMetadata", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "tasks", - "description": "", - "label": "repeated", - "type": "JobRunEventTask", - "longType": "JobRunEventTask", - "fullType": "mgmt.v1alpha1.JobRunEventTask", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRunEventMetadata", - "longName": "JobRunEventMetadata", - "fullName": "mgmt.v1alpha1.JobRunEventMetadata", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "sync_metadata", - "description": "", - "label": "", - "type": "JobRunSyncMetadata", - "longType": "JobRunSyncMetadata", - "fullType": "mgmt.v1alpha1.JobRunSyncMetadata", - "ismap": false, - "isoneof": true, - "oneofdecl": "metadata", - "defaultValue": "" - } - ] - }, - { - "name": "JobRunEventTask", - "longName": "JobRunEventTask", - "fullName": "mgmt.v1alpha1.JobRunEventTask", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "type", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "event_time", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "error", - "description": "", - "label": "", - "type": "JobRunEventTaskError", - "longType": "JobRunEventTaskError", - "fullType": "mgmt.v1alpha1.JobRunEventTaskError", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRunEventTaskError", - "longName": "JobRunEventTaskError", - "fullName": "mgmt.v1alpha1.JobRunEventTaskError", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "message", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "retry_state", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobRunSyncMetadata", - "longName": "JobRunSyncMetadata", - "fullName": "mgmt.v1alpha1.JobRunSyncMetadata", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobSource", - "longName": "JobSource", - "fullName": "mgmt.v1alpha1.JobSource", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "options", - "description": "", - "label": "", - "type": "JobSourceOptions", - "longType": "JobSourceOptions", - "fullType": "mgmt.v1alpha1.JobSourceOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "JobSourceOptions", - "longName": "JobSourceOptions", - "fullName": "mgmt.v1alpha1.JobSourceOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "postgres", - "description": "", - "label": "", - "type": "PostgresSourceConnectionOptions", - "longType": "PostgresSourceConnectionOptions", - "fullType": "mgmt.v1alpha1.PostgresSourceConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "aws_s3", - "description": "", - "label": "", - "type": "AwsS3SourceConnectionOptions", - "longType": "AwsS3SourceConnectionOptions", - "fullType": "mgmt.v1alpha1.AwsS3SourceConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "mysql", - "description": "", - "label": "", - "type": "MysqlSourceConnectionOptions", - "longType": "MysqlSourceConnectionOptions", - "fullType": "mgmt.v1alpha1.MysqlSourceConnectionOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - }, - { - "name": "generate", - "description": "", - "label": "", - "type": "GenerateSourceOptions", - "longType": "GenerateSourceOptions", - "fullType": "mgmt.v1alpha1.GenerateSourceOptions", - "ismap": false, - "isoneof": true, - "oneofdecl": "config", - "defaultValue": "" - } - ] - }, - { - "name": "JobSourceSqlSubetSchemas", - "longName": "JobSourceSqlSubetSchemas", - "fullName": "mgmt.v1alpha1.JobSourceSqlSubetSchemas", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "postgres_subset", - "description": "", - "label": "", - "type": "PostgresSourceSchemaSubset", - "longType": "PostgresSourceSchemaSubset", - "fullType": "mgmt.v1alpha1.PostgresSourceSchemaSubset", - "ismap": false, - "isoneof": true, - "oneofdecl": "schemas", - "defaultValue": "" - }, - { - "name": "mysql_subset", - "description": "", - "label": "", - "type": "MysqlSourceSchemaSubset", - "longType": "MysqlSourceSchemaSubset", - "fullType": "mgmt.v1alpha1.MysqlSourceSchemaSubset", - "ismap": false, - "isoneof": true, - "oneofdecl": "schemas", - "defaultValue": "" - } - ] - }, - { - "name": "JobStatusRecord", - "longName": "JobStatusRecord", - "fullName": "mgmt.v1alpha1.JobStatusRecord", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "status", - "description": "", - "label": "", - "type": "JobStatus", - "longType": "JobStatus", - "fullType": "mgmt.v1alpha1.JobStatus", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlDestinationConnectionOptions", - "longName": "MysqlDestinationConnectionOptions", - "fullName": "mgmt.v1alpha1.MysqlDestinationConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "truncate_table", - "description": "", - "label": "", - "type": "MysqlTruncateTableConfig", - "longType": "MysqlTruncateTableConfig", - "fullType": "mgmt.v1alpha1.MysqlTruncateTableConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "init_table_schema", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "on_conflict", - "description": "", - "label": "", - "type": "MysqlOnConflictConfig", - "longType": "MysqlOnConflictConfig", - "fullType": "mgmt.v1alpha1.MysqlOnConflictConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlOnConflictConfig", - "longName": "MysqlOnConflictConfig", - "fullName": "mgmt.v1alpha1.MysqlOnConflictConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "do_nothing", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlSourceConnectionOptions", - "longName": "MysqlSourceConnectionOptions", - "fullName": "mgmt.v1alpha1.MysqlSourceConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "halt_on_new_column_addition", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schemas", - "description": "", - "label": "repeated", - "type": "MysqlSourceSchemaOption", - "longType": "MysqlSourceSchemaOption", - "fullType": "mgmt.v1alpha1.MysqlSourceSchemaOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "subset_by_foreign_key_constraints", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlSourceSchemaOption", - "longName": "MysqlSourceSchemaOption", - "fullName": "mgmt.v1alpha1.MysqlSourceSchemaOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "tables", - "description": "", - "label": "repeated", - "type": "MysqlSourceTableOption", - "longType": "MysqlSourceTableOption", - "fullType": "mgmt.v1alpha1.MysqlSourceTableOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlSourceSchemaSubset", - "longName": "MysqlSourceSchemaSubset", - "fullName": "mgmt.v1alpha1.MysqlSourceSchemaSubset", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "mysql_schemas", - "description": "", - "label": "repeated", - "type": "MysqlSourceSchemaOption", - "longType": "MysqlSourceSchemaOption", - "fullType": "mgmt.v1alpha1.MysqlSourceSchemaOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlSourceTableOption", - "longName": "MysqlSourceTableOption", - "fullName": "mgmt.v1alpha1.MysqlSourceTableOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "where_clause", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_where_clause", - "defaultValue": "" - } - ] - }, - { - "name": "MysqlTruncateTableConfig", - "longName": "MysqlTruncateTableConfig", - "fullName": "mgmt.v1alpha1.MysqlTruncateTableConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "truncate_before_insert", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PauseJobRequest", - "longName": "PauseJobRequest", - "fullName": "mgmt.v1alpha1.PauseJobRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "pause", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "note", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_note", - "defaultValue": "" - } - ] - }, - { - "name": "PauseJobResponse", - "longName": "PauseJobResponse", - "fullName": "mgmt.v1alpha1.PauseJobResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PendingActivity", - "longName": "PendingActivity", - "fullName": "mgmt.v1alpha1.PendingActivity", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "status", - "description": "", - "label": "", - "type": "ActivityStatus", - "longType": "ActivityStatus", - "fullType": "mgmt.v1alpha1.ActivityStatus", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "activity_name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "last_failure", - "description": "", - "label": "optional", - "type": "ActivityFailure", - "longType": "ActivityFailure", - "fullType": "mgmt.v1alpha1.ActivityFailure", - "ismap": false, - "isoneof": true, - "oneofdecl": "_last_failure", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresDestinationConnectionOptions", - "longName": "PostgresDestinationConnectionOptions", - "fullName": "mgmt.v1alpha1.PostgresDestinationConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "truncate_table", - "description": "", - "label": "", - "type": "PostgresTruncateTableConfig", - "longType": "PostgresTruncateTableConfig", - "fullType": "mgmt.v1alpha1.PostgresTruncateTableConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "init_table_schema", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "on_conflict", - "description": "", - "label": "", - "type": "PostgresOnConflictConfig", - "longType": "PostgresOnConflictConfig", - "fullType": "mgmt.v1alpha1.PostgresOnConflictConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresOnConflictConfig", - "longName": "PostgresOnConflictConfig", - "fullName": "mgmt.v1alpha1.PostgresOnConflictConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "do_nothing", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresSourceConnectionOptions", - "longName": "PostgresSourceConnectionOptions", - "fullName": "mgmt.v1alpha1.PostgresSourceConnectionOptions", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "halt_on_new_column_addition", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schemas", - "description": "", - "label": "repeated", - "type": "PostgresSourceSchemaOption", - "longType": "PostgresSourceSchemaOption", - "fullType": "mgmt.v1alpha1.PostgresSourceSchemaOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "subset_by_foreign_key_constraints", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresSourceSchemaOption", - "longName": "PostgresSourceSchemaOption", - "fullName": "mgmt.v1alpha1.PostgresSourceSchemaOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "schema", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "tables", - "description": "", - "label": "repeated", - "type": "PostgresSourceTableOption", - "longType": "PostgresSourceTableOption", - "fullType": "mgmt.v1alpha1.PostgresSourceTableOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresSourceSchemaSubset", - "longName": "PostgresSourceSchemaSubset", - "fullName": "mgmt.v1alpha1.PostgresSourceSchemaSubset", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "postgres_schemas", - "description": "", - "label": "repeated", - "type": "PostgresSourceSchemaOption", - "longType": "PostgresSourceSchemaOption", - "fullType": "mgmt.v1alpha1.PostgresSourceSchemaOption", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresSourceTableOption", - "longName": "PostgresSourceTableOption", - "fullName": "mgmt.v1alpha1.PostgresSourceTableOption", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "table", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "where_clause", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_where_clause", - "defaultValue": "" - } - ] - }, - { - "name": "PostgresTruncateTableConfig", - "longName": "PostgresTruncateTableConfig", - "fullName": "mgmt.v1alpha1.PostgresTruncateTableConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "truncate_before_insert", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "cascade", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RetryPolicy", - "longName": "RetryPolicy", - "fullName": "mgmt.v1alpha1.RetryPolicy", - "description": "Defines the retry policy for an activity", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "maximum_attempts", - "description": "Maximum number of attempts. When exceeded the retries stop even if not expired yet.\nIf not set or set to 0, it means unlimited, and rely on activity ScheduleToCloseTimeout to stop.", - "label": "optional", - "type": "int32", - "longType": "int32", - "fullType": "int32", - "ismap": false, - "isoneof": true, - "oneofdecl": "_maximum_attempts", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobSourceSqlConnectionSubsetsRequest", - "longName": "SetJobSourceSqlConnectionSubsetsRequest", - "fullName": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "schemas", - "description": "", - "label": "", - "type": "JobSourceSqlSubetSchemas", - "longType": "JobSourceSqlSubetSchemas", - "fullType": "mgmt.v1alpha1.JobSourceSqlSubetSchemas", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "subset_by_foreign_key_constraints", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobSourceSqlConnectionSubsetsResponse", - "longName": "SetJobSourceSqlConnectionSubsetsResponse", - "fullName": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobSyncOptionsRequest", - "longName": "SetJobSyncOptionsRequest", - "fullName": "mgmt.v1alpha1.SetJobSyncOptionsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "The unique identifier of the job", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "sync_options", - "description": "The sync options object. The entire object must be provided and will fully overwrite the previous result", - "label": "", - "type": "ActivityOptions", - "longType": "ActivityOptions", - "fullType": "mgmt.v1alpha1.ActivityOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobSyncOptionsResponse", - "longName": "SetJobSyncOptionsResponse", - "fullName": "mgmt.v1alpha1.SetJobSyncOptionsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobWorkflowOptionsRequest", - "longName": "SetJobWorkflowOptionsRequest", - "fullName": "mgmt.v1alpha1.SetJobWorkflowOptionsRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "The unique identifier of the job", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "worfklow_options", - "description": "The workflow options object. The entire object must be provided and will fully overwrite the previous result", - "label": "", - "type": "WorkflowOptions", - "longType": "WorkflowOptions", - "fullType": "mgmt.v1alpha1.WorkflowOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetJobWorkflowOptionsResponse", - "longName": "SetJobWorkflowOptionsResponse", - "fullName": "mgmt.v1alpha1.SetJobWorkflowOptionsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TerminateJobRunRequest", - "longName": "TerminateJobRunRequest", - "fullName": "mgmt.v1alpha1.TerminateJobRunRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_run_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "TerminateJobRunResponse", - "longName": "TerminateJobRunResponse", - "fullName": "mgmt.v1alpha1.TerminateJobRunResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "UpdateJobDestinationConnectionRequest", - "longName": "UpdateJobDestinationConnectionRequest", - "fullName": "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "connection_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "options", - "description": "", - "label": "", - "type": "JobDestinationOptions", - "longType": "JobDestinationOptions", - "fullType": "mgmt.v1alpha1.JobDestinationOptions", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "destination_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateJobDestinationConnectionResponse", - "longName": "UpdateJobDestinationConnectionResponse", - "fullName": "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateJobScheduleRequest", - "longName": "UpdateJobScheduleRequest", - "fullName": "mgmt.v1alpha1.UpdateJobScheduleRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "cron_schedule", - "description": "", - "label": "optional", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "_cron_schedule", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateJobScheduleResponse", - "longName": "UpdateJobScheduleResponse", - "fullName": "mgmt.v1alpha1.UpdateJobScheduleResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateJobSourceConnectionRequest", - "longName": "UpdateJobSourceConnectionRequest", - "fullName": "mgmt.v1alpha1.UpdateJobSourceConnectionRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "source", - "description": "", - "label": "", - "type": "JobSource", - "longType": "JobSource", - "fullType": "mgmt.v1alpha1.JobSource", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "mappings", - "description": "", - "label": "repeated", - "type": "JobMapping", - "longType": "JobMapping", - "fullType": "mgmt.v1alpha1.JobMapping", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UpdateJobSourceConnectionResponse", - "longName": "UpdateJobSourceConnectionResponse", - "fullName": "mgmt.v1alpha1.UpdateJobSourceConnectionResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "job", - "description": "", - "label": "", - "type": "Job", - "longType": "Job", - "fullType": "mgmt.v1alpha1.Job", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "WorkflowOptions", - "longName": "WorkflowOptions", - "fullName": "mgmt.v1alpha1.WorkflowOptions", - "description": "Config that contains various timeouts that are configured in the underlying temporal workflow\nMore options will come in the future as needed", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "run_timeout", - "description": "The timeout for a single workflow run.\nMeasured in seconds", - "label": "optional", - "type": "int64", - "longType": "int64", - "fullType": "int64", - "ismap": false, - "isoneof": true, - "oneofdecl": "_run_timeout", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "JobService", - "longName": "JobService", - "fullName": "mgmt.v1alpha1.JobService", - "description": "", - "methods": [ - { - "name": "GetJobs", - "description": "", - "requestType": "GetJobsRequest", - "requestLongType": "GetJobsRequest", - "requestFullType": "mgmt.v1alpha1.GetJobsRequest", - "requestStreaming": false, - "responseType": "GetJobsResponse", - "responseLongType": "GetJobsResponse", - "responseFullType": "mgmt.v1alpha1.GetJobsResponse", - "responseStreaming": false - }, - { - "name": "GetJob", - "description": "", - "requestType": "GetJobRequest", - "requestLongType": "GetJobRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRequest", - "requestStreaming": false, - "responseType": "GetJobResponse", - "responseLongType": "GetJobResponse", - "responseFullType": "mgmt.v1alpha1.GetJobResponse", - "responseStreaming": false - }, - { - "name": "CreateJob", - "description": "", - "requestType": "CreateJobRequest", - "requestLongType": "CreateJobRequest", - "requestFullType": "mgmt.v1alpha1.CreateJobRequest", - "requestStreaming": false, - "responseType": "CreateJobResponse", - "responseLongType": "CreateJobResponse", - "responseFullType": "mgmt.v1alpha1.CreateJobResponse", - "responseStreaming": false - }, - { - "name": "DeleteJob", - "description": "", - "requestType": "DeleteJobRequest", - "requestLongType": "DeleteJobRequest", - "requestFullType": "mgmt.v1alpha1.DeleteJobRequest", - "requestStreaming": false, - "responseType": "DeleteJobResponse", - "responseLongType": "DeleteJobResponse", - "responseFullType": "mgmt.v1alpha1.DeleteJobResponse", - "responseStreaming": false - }, - { - "name": "IsJobNameAvailable", - "description": "", - "requestType": "IsJobNameAvailableRequest", - "requestLongType": "IsJobNameAvailableRequest", - "requestFullType": "mgmt.v1alpha1.IsJobNameAvailableRequest", - "requestStreaming": false, - "responseType": "IsJobNameAvailableResponse", - "responseLongType": "IsJobNameAvailableResponse", - "responseFullType": "mgmt.v1alpha1.IsJobNameAvailableResponse", - "responseStreaming": false - }, - { - "name": "UpdateJobSchedule", - "description": "", - "requestType": "UpdateJobScheduleRequest", - "requestLongType": "UpdateJobScheduleRequest", - "requestFullType": "mgmt.v1alpha1.UpdateJobScheduleRequest", - "requestStreaming": false, - "responseType": "UpdateJobScheduleResponse", - "responseLongType": "UpdateJobScheduleResponse", - "responseFullType": "mgmt.v1alpha1.UpdateJobScheduleResponse", - "responseStreaming": false - }, - { - "name": "UpdateJobSourceConnection", - "description": "", - "requestType": "UpdateJobSourceConnectionRequest", - "requestLongType": "UpdateJobSourceConnectionRequest", - "requestFullType": "mgmt.v1alpha1.UpdateJobSourceConnectionRequest", - "requestStreaming": false, - "responseType": "UpdateJobSourceConnectionResponse", - "responseLongType": "UpdateJobSourceConnectionResponse", - "responseFullType": "mgmt.v1alpha1.UpdateJobSourceConnectionResponse", - "responseStreaming": false - }, - { - "name": "SetJobSourceSqlConnectionSubsets", - "description": "", - "requestType": "SetJobSourceSqlConnectionSubsetsRequest", - "requestLongType": "SetJobSourceSqlConnectionSubsetsRequest", - "requestFullType": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest", - "requestStreaming": false, - "responseType": "SetJobSourceSqlConnectionSubsetsResponse", - "responseLongType": "SetJobSourceSqlConnectionSubsetsResponse", - "responseFullType": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse", - "responseStreaming": false - }, - { - "name": "UpdateJobDestinationConnection", - "description": "", - "requestType": "UpdateJobDestinationConnectionRequest", - "requestLongType": "UpdateJobDestinationConnectionRequest", - "requestFullType": "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest", - "requestStreaming": false, - "responseType": "UpdateJobDestinationConnectionResponse", - "responseLongType": "UpdateJobDestinationConnectionResponse", - "responseFullType": "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse", - "responseStreaming": false - }, - { - "name": "DeleteJobDestinationConnection", - "description": "", - "requestType": "DeleteJobDestinationConnectionRequest", - "requestLongType": "DeleteJobDestinationConnectionRequest", - "requestFullType": "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest", - "requestStreaming": false, - "responseType": "DeleteJobDestinationConnectionResponse", - "responseLongType": "DeleteJobDestinationConnectionResponse", - "responseFullType": "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse", - "responseStreaming": false - }, - { - "name": "CreateJobDestinationConnections", - "description": "", - "requestType": "CreateJobDestinationConnectionsRequest", - "requestLongType": "CreateJobDestinationConnectionsRequest", - "requestFullType": "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest", - "requestStreaming": false, - "responseType": "CreateJobDestinationConnectionsResponse", - "responseLongType": "CreateJobDestinationConnectionsResponse", - "responseFullType": "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse", - "responseStreaming": false - }, - { - "name": "PauseJob", - "description": "", - "requestType": "PauseJobRequest", - "requestLongType": "PauseJobRequest", - "requestFullType": "mgmt.v1alpha1.PauseJobRequest", - "requestStreaming": false, - "responseType": "PauseJobResponse", - "responseLongType": "PauseJobResponse", - "responseFullType": "mgmt.v1alpha1.PauseJobResponse", - "responseStreaming": false - }, - { - "name": "GetJobRecentRuns", - "description": "Returns a list of recently invoked job runs baseds on the Temporal cron scheduler. This will return a list of job runs that include archived runs", - "requestType": "GetJobRecentRunsRequest", - "requestLongType": "GetJobRecentRunsRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRecentRunsRequest", - "requestStreaming": false, - "responseType": "GetJobRecentRunsResponse", - "responseLongType": "GetJobRecentRunsResponse", - "responseFullType": "mgmt.v1alpha1.GetJobRecentRunsResponse", - "responseStreaming": false - }, - { - "name": "GetJobNextRuns", - "description": "Returns a list of runs that are scheduled for execution based on the Temporal cron scheduler.", - "requestType": "GetJobNextRunsRequest", - "requestLongType": "GetJobNextRunsRequest", - "requestFullType": "mgmt.v1alpha1.GetJobNextRunsRequest", - "requestStreaming": false, - "responseType": "GetJobNextRunsResponse", - "responseLongType": "GetJobNextRunsResponse", - "responseFullType": "mgmt.v1alpha1.GetJobNextRunsResponse", - "responseStreaming": false - }, - { - "name": "GetJobStatus", - "description": "", - "requestType": "GetJobStatusRequest", - "requestLongType": "GetJobStatusRequest", - "requestFullType": "mgmt.v1alpha1.GetJobStatusRequest", - "requestStreaming": false, - "responseType": "GetJobStatusResponse", - "responseLongType": "GetJobStatusResponse", - "responseFullType": "mgmt.v1alpha1.GetJobStatusResponse", - "responseStreaming": false - }, - { - "name": "GetJobStatuses", - "description": "", - "requestType": "GetJobStatusesRequest", - "requestLongType": "GetJobStatusesRequest", - "requestFullType": "mgmt.v1alpha1.GetJobStatusesRequest", - "requestStreaming": false, - "responseType": "GetJobStatusesResponse", - "responseLongType": "GetJobStatusesResponse", - "responseFullType": "mgmt.v1alpha1.GetJobStatusesResponse", - "responseStreaming": false - }, - { - "name": "GetJobRuns", - "description": "Returns a list of job runs by either account or job", - "requestType": "GetJobRunsRequest", - "requestLongType": "GetJobRunsRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRunsRequest", - "requestStreaming": false, - "responseType": "GetJobRunsResponse", - "responseLongType": "GetJobRunsResponse", - "responseFullType": "mgmt.v1alpha1.GetJobRunsResponse", - "responseStreaming": false - }, - { - "name": "GetJobRunEvents", - "description": "", - "requestType": "GetJobRunEventsRequest", - "requestLongType": "GetJobRunEventsRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRunEventsRequest", - "requestStreaming": false, - "responseType": "GetJobRunEventsResponse", - "responseLongType": "GetJobRunEventsResponse", - "responseFullType": "mgmt.v1alpha1.GetJobRunEventsResponse", - "responseStreaming": false - }, - { - "name": "GetJobRun", - "description": "Returns a specific job run, along with any of its pending activities", - "requestType": "GetJobRunRequest", - "requestLongType": "GetJobRunRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRunRequest", - "requestStreaming": false, - "responseType": "GetJobRunResponse", - "responseLongType": "GetJobRunResponse", - "responseFullType": "mgmt.v1alpha1.GetJobRunResponse", - "responseStreaming": false - }, - { - "name": "DeleteJobRun", - "description": "", - "requestType": "DeleteJobRunRequest", - "requestLongType": "DeleteJobRunRequest", - "requestFullType": "mgmt.v1alpha1.DeleteJobRunRequest", - "requestStreaming": false, - "responseType": "DeleteJobRunResponse", - "responseLongType": "DeleteJobRunResponse", - "responseFullType": "mgmt.v1alpha1.DeleteJobRunResponse", - "responseStreaming": false - }, - { - "name": "CreateJobRun", - "description": "", - "requestType": "CreateJobRunRequest", - "requestLongType": "CreateJobRunRequest", - "requestFullType": "mgmt.v1alpha1.CreateJobRunRequest", - "requestStreaming": false, - "responseType": "CreateJobRunResponse", - "responseLongType": "CreateJobRunResponse", - "responseFullType": "mgmt.v1alpha1.CreateJobRunResponse", - "responseStreaming": false - }, - { - "name": "CancelJobRun", - "description": "", - "requestType": "CancelJobRunRequest", - "requestLongType": "CancelJobRunRequest", - "requestFullType": "mgmt.v1alpha1.CancelJobRunRequest", - "requestStreaming": false, - "responseType": "CancelJobRunResponse", - "responseLongType": "CancelJobRunResponse", - "responseFullType": "mgmt.v1alpha1.CancelJobRunResponse", - "responseStreaming": false - }, - { - "name": "TerminateJobRun", - "description": "", - "requestType": "TerminateJobRunRequest", - "requestLongType": "TerminateJobRunRequest", - "requestFullType": "mgmt.v1alpha1.TerminateJobRunRequest", - "requestStreaming": false, - "responseType": "TerminateJobRunResponse", - "responseLongType": "TerminateJobRunResponse", - "responseFullType": "mgmt.v1alpha1.TerminateJobRunResponse", - "responseStreaming": false - }, - { - "name": "GetJobRunLogsStream", - "description": "Returns a stream of logs from the worker nodes that pertain to a specific job run", - "requestType": "GetJobRunLogsStreamRequest", - "requestLongType": "GetJobRunLogsStreamRequest", - "requestFullType": "mgmt.v1alpha1.GetJobRunLogsStreamRequest", - "requestStreaming": false, - "responseType": "GetJobRunLogsStreamResponse", - "responseLongType": "GetJobRunLogsStreamResponse", - "responseFullType": "mgmt.v1alpha1.GetJobRunLogsStreamResponse", - "responseStreaming": true - }, - { - "name": "SetJobWorkflowOptions", - "description": "Set any job workflow options. Must provide entire object as is it will fully override the previous configuration", - "requestType": "SetJobWorkflowOptionsRequest", - "requestLongType": "SetJobWorkflowOptionsRequest", - "requestFullType": "mgmt.v1alpha1.SetJobWorkflowOptionsRequest", - "requestStreaming": false, - "responseType": "SetJobWorkflowOptionsResponse", - "responseLongType": "SetJobWorkflowOptionsResponse", - "responseFullType": "mgmt.v1alpha1.SetJobWorkflowOptionsResponse", - "responseStreaming": false - }, - { - "name": "SetJobSyncOptions", - "description": "Set the job sync options. Must provide entire object as it will fully override the previous configuration", - "requestType": "SetJobSyncOptionsRequest", - "requestLongType": "SetJobSyncOptionsRequest", - "requestFullType": "mgmt.v1alpha1.SetJobSyncOptionsRequest", - "requestStreaming": false, - "responseType": "SetJobSyncOptionsResponse", - "responseLongType": "SetJobSyncOptionsResponse", - "responseFullType": "mgmt.v1alpha1.SetJobSyncOptionsResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/metrics.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": true, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [ - { - "name": "RangedMetricName", - "longName": "RangedMetricName", - "fullName": "mgmt.v1alpha1.RangedMetricName", - "description": "", - "values": [ - { - "name": "RANGED_METRIC_NAME_UNSPECIFIED", - "number": "0", - "description": "If unspecified, an error will be thrown" - }, - { - "name": "RANGED_METRIC_NAME_INPUT_RECEIVED", - "number": "1", - "description": "The input_received metric" - } - ] - } - ], - "extensions": [], - "messages": [ - { - "name": "Date", - "longName": "Date", - "fullName": "mgmt.v1alpha1.Date", - "description": "Represents a whole or partial calendar date, such as a birthday. The time of\nday and time zone are either specified elsewhere or are insignificant. The\ndate is relative to the Gregorian Calendar. This can represent one of the\nfollowing:\n\n* A full date, with non-zero year, month, and day values\n* A month and day value, with a zero year, such as an anniversary\n* A year on its own, with zero month and day values\n* A year and month value, with a zero day, such as a credit card expiration\ndate\n\nRelated types are [google.type.TimeOfDay][google.type.TimeOfDay] and\n`google.protobuf.Timestamp`.", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "year", - "description": "Year of the date. Must be from 1 to 9999, or 0 to specify a date without\na year.", - "label": "", - "type": "uint32", - "longType": "uint32", - "fullType": "uint32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "month", - "description": "Month of a year. Must be from 1 to 12, or 0 to specify a year without a\nmonth and day.", - "label": "", - "type": "uint32", - "longType": "uint32", - "fullType": "uint32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "day", - "description": "Day of a month. Must be from 1 to 31 and valid for the year and month, or 0\nto specify a year by itself or a year and month where the day isn't\nsignificant.", - "label": "", - "type": "uint32", - "longType": "uint32", - "fullType": "uint32", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "DayResult", - "longName": "DayResult", - "fullName": "mgmt.v1alpha1.DayResult", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "date", - "description": "", - "label": "", - "type": "Date", - "longType": "Date", - "fullType": "mgmt.v1alpha1.Date", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "count", - "description": "", - "label": "", - "type": "uint64", - "longType": "uint64", - "fullType": "uint64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetDailyMetricCountRequest", - "longName": "GetDailyMetricCountRequest", - "fullName": "mgmt.v1alpha1.GetDailyMetricCountRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "start", - "description": "The start day", - "label": "", - "type": "Date", - "longType": "Date", - "fullType": "mgmt.v1alpha1.Date", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "end", - "description": "The end day", - "label": "", - "type": "Date", - "longType": "Date", - "fullType": "mgmt.v1alpha1.Date", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "metric", - "description": "The metric to return", - "label": "", - "type": "RangedMetricName", - "longType": "RangedMetricName", - "fullType": "mgmt.v1alpha1.RangedMetricName", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "The account identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - }, - { - "name": "job_id", - "description": "The job identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - }, - { - "name": "run_id", - "description": "The run identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - } - ] - }, - { - "name": "GetDailyMetricCountResponse", - "longName": "GetDailyMetricCountResponse", - "fullName": "mgmt.v1alpha1.GetDailyMetricCountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "results", - "description": "", - "label": "repeated", - "type": "DayResult", - "longType": "DayResult", - "fullType": "mgmt.v1alpha1.DayResult", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetMetricCountRequest", - "longName": "GetMetricCountRequest", - "fullName": "mgmt.v1alpha1.GetMetricCountRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": true, - "extensions": [], - "fields": [ - { - "name": "start", - "description": "The start time", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "end", - "description": "The end time", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "metric", - "description": "The metric to return", - "label": "", - "type": "RangedMetricName", - "longType": "RangedMetricName", - "fullType": "mgmt.v1alpha1.RangedMetricName", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "The account identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - }, - { - "name": "job_id", - "description": "The job identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - }, - { - "name": "run_id", - "description": "The run identifier that will be used to filter by", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": true, - "oneofdecl": "identifier", - "defaultValue": "" - } - ] - }, - { - "name": "GetMetricCountResponse", - "longName": "GetMetricCountResponse", - "fullName": "mgmt.v1alpha1.GetMetricCountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "count", - "description": "The summed up count of the metric based on the input query and timerange specified", - "label": "", - "type": "uint64", - "longType": "uint64", - "fullType": "uint64", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "MetricsService", - "longName": "MetricsService", - "fullName": "mgmt.v1alpha1.MetricsService", - "description": "", - "methods": [ - { - "name": "GetDailyMetricCount", - "description": "Retrieve a timed range of records", - "requestType": "GetDailyMetricCountRequest", - "requestLongType": "GetDailyMetricCountRequest", - "requestFullType": "mgmt.v1alpha1.GetDailyMetricCountRequest", - "requestStreaming": false, - "responseType": "GetDailyMetricCountResponse", - "responseLongType": "GetDailyMetricCountResponse", - "responseFullType": "mgmt.v1alpha1.GetDailyMetricCountResponse", - "responseStreaming": false - }, - { - "name": "GetMetricCount", - "description": "For the given metric and time range, returns the total count found", - "requestType": "GetMetricCountRequest", - "requestLongType": "GetMetricCountRequest", - "requestFullType": "mgmt.v1alpha1.GetMetricCountRequest", - "requestStreaming": false, - "responseType": "GetMetricCountResponse", - "responseLongType": "GetMetricCountResponse", - "responseFullType": "mgmt.v1alpha1.GetMetricCountResponse", - "responseStreaming": false - } - ] - } - ] - }, - { - "name": "mgmt/v1alpha1/user_account.proto", - "description": "", - "package": "mgmt.v1alpha1", - "hasEnums": true, - "hasExtensions": false, - "hasMessages": true, - "hasServices": true, - "enums": [ - { - "name": "UserAccountType", - "longName": "UserAccountType", - "fullName": "mgmt.v1alpha1.UserAccountType", - "description": "", - "values": [ - { - "name": "USER_ACCOUNT_TYPE_UNSPECIFIED", - "number": "0", - "description": "" - }, - { - "name": "USER_ACCOUNT_TYPE_PERSONAL", - "number": "1", - "description": "" - }, - { - "name": "USER_ACCOUNT_TYPE_TEAM", - "number": "2", - "description": "" - }, - { - "name": "USER_ACCOUNT_TYPE_ENTERPRISE", - "number": "3", - "description": "" - } - ] - } - ], - "extensions": [], - "messages": [ - { - "name": "AcceptTeamAccountInviteRequest", - "longName": "AcceptTeamAccountInviteRequest", - "fullName": "mgmt.v1alpha1.AcceptTeamAccountInviteRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "token", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AcceptTeamAccountInviteResponse", - "longName": "AcceptTeamAccountInviteResponse", - "fullName": "mgmt.v1alpha1.AcceptTeamAccountInviteResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account", - "description": "", - "label": "", - "type": "UserAccount", - "longType": "UserAccount", - "fullType": "mgmt.v1alpha1.UserAccount", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AccountInvite", - "longName": "AccountInvite", - "fullName": "mgmt.v1alpha1.AccountInvite", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "sender_user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "email", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "token", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "accepted", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "created_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "updated_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "expires_at", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AccountOnboardingConfig", - "longName": "AccountOnboardingConfig", - "fullName": "mgmt.v1alpha1.AccountOnboardingConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "has_created_source_connection", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "has_created_destination_connection", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "has_created_job", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "has_invited_members", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AccountTemporalConfig", - "longName": "AccountTemporalConfig", - "fullName": "mgmt.v1alpha1.AccountTemporalConfig", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "url", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "namespace", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "sync_job_queue_name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "AccountUser", - "longName": "AccountUser", - "fullName": "mgmt.v1alpha1.AccountUser", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "image", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "email", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "ConvertPersonalToTeamAccountRequest", - "longName": "ConvertPersonalToTeamAccountRequest", - "fullName": "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "ConvertPersonalToTeamAccountResponse", - "longName": "ConvertPersonalToTeamAccountResponse", - "fullName": "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "CreateTeamAccountRequest", - "longName": "CreateTeamAccountRequest", - "fullName": "mgmt.v1alpha1.CreateTeamAccountRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "CreateTeamAccountResponse", - "longName": "CreateTeamAccountResponse", - "fullName": "mgmt.v1alpha1.CreateTeamAccountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountOnboardingConfigRequest", - "longName": "GetAccountOnboardingConfigRequest", - "fullName": "mgmt.v1alpha1.GetAccountOnboardingConfigRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountOnboardingConfigResponse", - "longName": "GetAccountOnboardingConfigResponse", - "fullName": "mgmt.v1alpha1.GetAccountOnboardingConfigResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "config", - "description": "", - "label": "", - "type": "AccountOnboardingConfig", - "longType": "AccountOnboardingConfig", - "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountTemporalConfigRequest", - "longName": "GetAccountTemporalConfigRequest", - "fullName": "mgmt.v1alpha1.GetAccountTemporalConfigRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetAccountTemporalConfigResponse", - "longName": "GetAccountTemporalConfigResponse", - "fullName": "mgmt.v1alpha1.GetAccountTemporalConfigResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "config", - "description": "", - "label": "", - "type": "AccountTemporalConfig", - "longType": "AccountTemporalConfig", - "fullType": "mgmt.v1alpha1.AccountTemporalConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetSystemInformationRequest", - "longName": "GetSystemInformationRequest", - "fullName": "mgmt.v1alpha1.GetSystemInformationRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetSystemInformationResponse", - "longName": "GetSystemInformationResponse", - "fullName": "mgmt.v1alpha1.GetSystemInformationResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "version", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "commit", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "compiler", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "platform", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "build_date", - "description": "", - "label": "", - "type": "Timestamp", - "longType": "google.protobuf.Timestamp", - "fullType": "google.protobuf.Timestamp", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetTeamAccountInvitesRequest", - "longName": "GetTeamAccountInvitesRequest", - "fullName": "mgmt.v1alpha1.GetTeamAccountInvitesRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetTeamAccountInvitesResponse", - "longName": "GetTeamAccountInvitesResponse", - "fullName": "mgmt.v1alpha1.GetTeamAccountInvitesResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "invites", - "description": "", - "label": "repeated", - "type": "AccountInvite", - "longType": "AccountInvite", - "fullType": "mgmt.v1alpha1.AccountInvite", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetTeamAccountMembersRequest", - "longName": "GetTeamAccountMembersRequest", - "fullName": "mgmt.v1alpha1.GetTeamAccountMembersRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetTeamAccountMembersResponse", - "longName": "GetTeamAccountMembersResponse", - "fullName": "mgmt.v1alpha1.GetTeamAccountMembersResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "users", - "description": "", - "label": "repeated", - "type": "AccountUser", - "longType": "AccountUser", - "fullType": "mgmt.v1alpha1.AccountUser", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserAccountsRequest", - "longName": "GetUserAccountsRequest", - "fullName": "mgmt.v1alpha1.GetUserAccountsRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetUserAccountsResponse", - "longName": "GetUserAccountsResponse", - "fullName": "mgmt.v1alpha1.GetUserAccountsResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "accounts", - "description": "", - "label": "repeated", - "type": "UserAccount", - "longType": "UserAccount", - "fullType": "mgmt.v1alpha1.UserAccount", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "GetUserRequest", - "longName": "GetUserRequest", - "fullName": "mgmt.v1alpha1.GetUserRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "GetUserResponse", - "longName": "GetUserResponse", - "fullName": "mgmt.v1alpha1.GetUserResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "InviteUserToTeamAccountRequest", - "longName": "InviteUserToTeamAccountRequest", - "fullName": "mgmt.v1alpha1.InviteUserToTeamAccountRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "email", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "InviteUserToTeamAccountResponse", - "longName": "InviteUserToTeamAccountResponse", - "fullName": "mgmt.v1alpha1.InviteUserToTeamAccountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "invite", - "description": "", - "label": "", - "type": "AccountInvite", - "longType": "AccountInvite", - "fullType": "mgmt.v1alpha1.AccountInvite", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsUserInAccountRequest", - "longName": "IsUserInAccountRequest", - "fullName": "mgmt.v1alpha1.IsUserInAccountRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "IsUserInAccountResponse", - "longName": "IsUserInAccountResponse", - "fullName": "mgmt.v1alpha1.IsUserInAccountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "ok", - "description": "", - "label": "", - "type": "bool", - "longType": "bool", - "fullType": "bool", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RemoveTeamAccountInviteRequest", - "longName": "RemoveTeamAccountInviteRequest", - "fullName": "mgmt.v1alpha1.RemoveTeamAccountInviteRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RemoveTeamAccountInviteResponse", - "longName": "RemoveTeamAccountInviteResponse", - "fullName": "mgmt.v1alpha1.RemoveTeamAccountInviteResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "RemoveTeamAccountMemberRequest", - "longName": "RemoveTeamAccountMemberRequest", - "fullName": "mgmt.v1alpha1.RemoveTeamAccountMemberRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "RemoveTeamAccountMemberResponse", - "longName": "RemoveTeamAccountMemberResponse", - "fullName": "mgmt.v1alpha1.RemoveTeamAccountMemberResponse", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "SetAccountOnboardingConfigRequest", - "longName": "SetAccountOnboardingConfigRequest", - "fullName": "mgmt.v1alpha1.SetAccountOnboardingConfigRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "config", - "description": "", - "label": "", - "type": "AccountOnboardingConfig", - "longType": "AccountOnboardingConfig", - "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetAccountOnboardingConfigResponse", - "longName": "SetAccountOnboardingConfigResponse", - "fullName": "mgmt.v1alpha1.SetAccountOnboardingConfigResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "config", - "description": "", - "label": "", - "type": "AccountOnboardingConfig", - "longType": "AccountOnboardingConfig", - "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetAccountTemporalConfigRequest", - "longName": "SetAccountTemporalConfigRequest", - "fullName": "mgmt.v1alpha1.SetAccountTemporalConfigRequest", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "config", - "description": "", - "label": "", - "type": "AccountTemporalConfig", - "longType": "AccountTemporalConfig", - "fullType": "mgmt.v1alpha1.AccountTemporalConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetAccountTemporalConfigResponse", - "longName": "SetAccountTemporalConfigResponse", - "fullName": "mgmt.v1alpha1.SetAccountTemporalConfigResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "config", - "description": "", - "label": "", - "type": "AccountTemporalConfig", - "longType": "AccountTemporalConfig", - "fullType": "mgmt.v1alpha1.AccountTemporalConfig", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetPersonalAccountRequest", - "longName": "SetPersonalAccountRequest", - "fullName": "mgmt.v1alpha1.SetPersonalAccountRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "SetPersonalAccountResponse", - "longName": "SetPersonalAccountResponse", - "fullName": "mgmt.v1alpha1.SetPersonalAccountResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "account_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "SetUserRequest", - "longName": "SetUserRequest", - "fullName": "mgmt.v1alpha1.SetUserRequest", - "description": "", - "hasExtensions": false, - "hasFields": false, - "hasOneofs": false, - "extensions": [], - "fields": [] - }, - { - "name": "SetUserResponse", - "longName": "SetUserResponse", - "fullName": "mgmt.v1alpha1.SetUserResponse", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "user_id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - }, - { - "name": "UserAccount", - "longName": "UserAccount", - "fullName": "mgmt.v1alpha1.UserAccount", - "description": "", - "hasExtensions": false, - "hasFields": true, - "hasOneofs": false, - "extensions": [], - "fields": [ - { - "name": "id", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "name", - "description": "", - "label": "", - "type": "string", - "longType": "string", - "fullType": "string", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - }, - { - "name": "type", - "description": "", - "label": "", - "type": "UserAccountType", - "longType": "UserAccountType", - "fullType": "mgmt.v1alpha1.UserAccountType", - "ismap": false, - "isoneof": false, - "oneofdecl": "", - "defaultValue": "" - } - ] - } - ], - "services": [ - { - "name": "UserAccountService", - "longName": "UserAccountService", - "fullName": "mgmt.v1alpha1.UserAccountService", - "description": "", - "methods": [ - { - "name": "GetUser", - "description": "", - "requestType": "GetUserRequest", - "requestLongType": "GetUserRequest", - "requestFullType": "mgmt.v1alpha1.GetUserRequest", - "requestStreaming": false, - "responseType": "GetUserResponse", - "responseLongType": "GetUserResponse", - "responseFullType": "mgmt.v1alpha1.GetUserResponse", - "responseStreaming": false - }, - { - "name": "SetUser", - "description": "", - "requestType": "SetUserRequest", - "requestLongType": "SetUserRequest", - "requestFullType": "mgmt.v1alpha1.SetUserRequest", - "requestStreaming": false, - "responseType": "SetUserResponse", - "responseLongType": "SetUserResponse", - "responseFullType": "mgmt.v1alpha1.SetUserResponse", - "responseStreaming": false - }, - { - "name": "GetUserAccounts", - "description": "", - "requestType": "GetUserAccountsRequest", - "requestLongType": "GetUserAccountsRequest", - "requestFullType": "mgmt.v1alpha1.GetUserAccountsRequest", - "requestStreaming": false, - "responseType": "GetUserAccountsResponse", - "responseLongType": "GetUserAccountsResponse", - "responseFullType": "mgmt.v1alpha1.GetUserAccountsResponse", - "responseStreaming": false - }, - { - "name": "SetPersonalAccount", - "description": "", - "requestType": "SetPersonalAccountRequest", - "requestLongType": "SetPersonalAccountRequest", - "requestFullType": "mgmt.v1alpha1.SetPersonalAccountRequest", - "requestStreaming": false, - "responseType": "SetPersonalAccountResponse", - "responseLongType": "SetPersonalAccountResponse", - "responseFullType": "mgmt.v1alpha1.SetPersonalAccountResponse", - "responseStreaming": false - }, - { - "name": "ConvertPersonalToTeamAccount", - "description": "", - "requestType": "ConvertPersonalToTeamAccountRequest", - "requestLongType": "ConvertPersonalToTeamAccountRequest", - "requestFullType": "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest", - "requestStreaming": false, - "responseType": "ConvertPersonalToTeamAccountResponse", - "responseLongType": "ConvertPersonalToTeamAccountResponse", - "responseFullType": "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse", - "responseStreaming": false - }, - { - "name": "CreateTeamAccount", - "description": "", - "requestType": "CreateTeamAccountRequest", - "requestLongType": "CreateTeamAccountRequest", - "requestFullType": "mgmt.v1alpha1.CreateTeamAccountRequest", - "requestStreaming": false, - "responseType": "CreateTeamAccountResponse", - "responseLongType": "CreateTeamAccountResponse", - "responseFullType": "mgmt.v1alpha1.CreateTeamAccountResponse", - "responseStreaming": false - }, - { - "name": "IsUserInAccount", - "description": "", - "requestType": "IsUserInAccountRequest", - "requestLongType": "IsUserInAccountRequest", - "requestFullType": "mgmt.v1alpha1.IsUserInAccountRequest", - "requestStreaming": false, - "responseType": "IsUserInAccountResponse", - "responseLongType": "IsUserInAccountResponse", - "responseFullType": "mgmt.v1alpha1.IsUserInAccountResponse", - "responseStreaming": false - }, - { - "name": "GetAccountTemporalConfig", - "description": "", - "requestType": "GetAccountTemporalConfigRequest", - "requestLongType": "GetAccountTemporalConfigRequest", - "requestFullType": "mgmt.v1alpha1.GetAccountTemporalConfigRequest", - "requestStreaming": false, - "responseType": "GetAccountTemporalConfigResponse", - "responseLongType": "GetAccountTemporalConfigResponse", - "responseFullType": "mgmt.v1alpha1.GetAccountTemporalConfigResponse", - "responseStreaming": false - }, - { - "name": "SetAccountTemporalConfig", - "description": "", - "requestType": "SetAccountTemporalConfigRequest", - "requestLongType": "SetAccountTemporalConfigRequest", - "requestFullType": "mgmt.v1alpha1.SetAccountTemporalConfigRequest", - "requestStreaming": false, - "responseType": "SetAccountTemporalConfigResponse", - "responseLongType": "SetAccountTemporalConfigResponse", - "responseFullType": "mgmt.v1alpha1.SetAccountTemporalConfigResponse", - "responseStreaming": false - }, - { - "name": "GetTeamAccountMembers", - "description": "", - "requestType": "GetTeamAccountMembersRequest", - "requestLongType": "GetTeamAccountMembersRequest", - "requestFullType": "mgmt.v1alpha1.GetTeamAccountMembersRequest", - "requestStreaming": false, - "responseType": "GetTeamAccountMembersResponse", - "responseLongType": "GetTeamAccountMembersResponse", - "responseFullType": "mgmt.v1alpha1.GetTeamAccountMembersResponse", - "responseStreaming": false - }, - { - "name": "RemoveTeamAccountMember", - "description": "", - "requestType": "RemoveTeamAccountMemberRequest", - "requestLongType": "RemoveTeamAccountMemberRequest", - "requestFullType": "mgmt.v1alpha1.RemoveTeamAccountMemberRequest", - "requestStreaming": false, - "responseType": "RemoveTeamAccountMemberResponse", - "responseLongType": "RemoveTeamAccountMemberResponse", - "responseFullType": "mgmt.v1alpha1.RemoveTeamAccountMemberResponse", - "responseStreaming": false - }, - { - "name": "InviteUserToTeamAccount", - "description": "", - "requestType": "InviteUserToTeamAccountRequest", - "requestLongType": "InviteUserToTeamAccountRequest", - "requestFullType": "mgmt.v1alpha1.InviteUserToTeamAccountRequest", - "requestStreaming": false, - "responseType": "InviteUserToTeamAccountResponse", - "responseLongType": "InviteUserToTeamAccountResponse", - "responseFullType": "mgmt.v1alpha1.InviteUserToTeamAccountResponse", - "responseStreaming": false - }, - { - "name": "GetTeamAccountInvites", - "description": "", - "requestType": "GetTeamAccountInvitesRequest", - "requestLongType": "GetTeamAccountInvitesRequest", - "requestFullType": "mgmt.v1alpha1.GetTeamAccountInvitesRequest", - "requestStreaming": false, - "responseType": "GetTeamAccountInvitesResponse", - "responseLongType": "GetTeamAccountInvitesResponse", - "responseFullType": "mgmt.v1alpha1.GetTeamAccountInvitesResponse", - "responseStreaming": false - }, - { - "name": "RemoveTeamAccountInvite", - "description": "", - "requestType": "RemoveTeamAccountInviteRequest", - "requestLongType": "RemoveTeamAccountInviteRequest", - "requestFullType": "mgmt.v1alpha1.RemoveTeamAccountInviteRequest", - "requestStreaming": false, - "responseType": "RemoveTeamAccountInviteResponse", - "responseLongType": "RemoveTeamAccountInviteResponse", - "responseFullType": "mgmt.v1alpha1.RemoveTeamAccountInviteResponse", - "responseStreaming": false - }, - { - "name": "AcceptTeamAccountInvite", - "description": "", - "requestType": "AcceptTeamAccountInviteRequest", - "requestLongType": "AcceptTeamAccountInviteRequest", - "requestFullType": "mgmt.v1alpha1.AcceptTeamAccountInviteRequest", - "requestStreaming": false, - "responseType": "AcceptTeamAccountInviteResponse", - "responseLongType": "AcceptTeamAccountInviteResponse", - "responseFullType": "mgmt.v1alpha1.AcceptTeamAccountInviteResponse", - "responseStreaming": false - }, - { - "name": "GetSystemInformation", - "description": "", - "requestType": "GetSystemInformationRequest", - "requestLongType": "GetSystemInformationRequest", - "requestFullType": "mgmt.v1alpha1.GetSystemInformationRequest", - "requestStreaming": false, - "responseType": "GetSystemInformationResponse", - "responseLongType": "GetSystemInformationResponse", - "responseFullType": "mgmt.v1alpha1.GetSystemInformationResponse", - "responseStreaming": false - }, - { - "name": "GetAccountOnboardingConfig", - "description": "", - "requestType": "GetAccountOnboardingConfigRequest", - "requestLongType": "GetAccountOnboardingConfigRequest", - "requestFullType": "mgmt.v1alpha1.GetAccountOnboardingConfigRequest", - "requestStreaming": false, - "responseType": "GetAccountOnboardingConfigResponse", - "responseLongType": "GetAccountOnboardingConfigResponse", - "responseFullType": "mgmt.v1alpha1.GetAccountOnboardingConfigResponse", - "responseStreaming": false - }, - { - "name": "SetAccountOnboardingConfig", - "description": "", - "requestType": "SetAccountOnboardingConfigRequest", - "requestLongType": "SetAccountOnboardingConfigRequest", - "requestFullType": "mgmt.v1alpha1.SetAccountOnboardingConfigRequest", - "requestStreaming": false, - "responseType": "SetAccountOnboardingConfigResponse", - "responseLongType": "SetAccountOnboardingConfigResponse", - "responseFullType": "mgmt.v1alpha1.SetAccountOnboardingConfigResponse", - "responseStreaming": false - } - ] - } - ] - } - ], - "scalarValueTypes": [ - { - "protoType": "double", - "notes": "", - "cppType": "double", - "csType": "double", - "goType": "float64", - "javaType": "double", - "phpType": "float", - "pythonType": "float", - "rubyType": "Float" - }, - { - "protoType": "float", - "notes": "", - "cppType": "float", - "csType": "float", - "goType": "float32", - "javaType": "float", - "phpType": "float", - "pythonType": "float", - "rubyType": "Float" - }, - { - "protoType": "int32", - "notes": "Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint32 instead.", - "cppType": "int32", - "csType": "int", - "goType": "int32", - "javaType": "int", - "phpType": "integer", - "pythonType": "int", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "int64", - "notes": "Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint64 instead.", - "cppType": "int64", - "csType": "long", - "goType": "int64", - "javaType": "long", - "phpType": "integer/string", - "pythonType": "int/long", - "rubyType": "Bignum" - }, - { - "protoType": "uint32", - "notes": "Uses variable-length encoding.", - "cppType": "uint32", - "csType": "uint", - "goType": "uint32", - "javaType": "int", - "phpType": "integer", - "pythonType": "int/long", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "uint64", - "notes": "Uses variable-length encoding.", - "cppType": "uint64", - "csType": "ulong", - "goType": "uint64", - "javaType": "long", - "phpType": "integer/string", - "pythonType": "int/long", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "sint32", - "notes": "Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int32s.", - "cppType": "int32", - "csType": "int", - "goType": "int32", - "javaType": "int", - "phpType": "integer", - "pythonType": "int", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "sint64", - "notes": "Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int64s.", - "cppType": "int64", - "csType": "long", - "goType": "int64", - "javaType": "long", - "phpType": "integer/string", - "pythonType": "int/long", - "rubyType": "Bignum" - }, - { - "protoType": "fixed32", - "notes": "Always four bytes. More efficient than uint32 if values are often greater than 2^28.", - "cppType": "uint32", - "csType": "uint", - "goType": "uint32", - "javaType": "int", - "phpType": "integer", - "pythonType": "int", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "fixed64", - "notes": "Always eight bytes. More efficient than uint64 if values are often greater than 2^56.", - "cppType": "uint64", - "csType": "ulong", - "goType": "uint64", - "javaType": "long", - "phpType": "integer/string", - "pythonType": "int/long", - "rubyType": "Bignum" - }, - { - "protoType": "sfixed32", - "notes": "Always four bytes.", - "cppType": "int32", - "csType": "int", - "goType": "int32", - "javaType": "int", - "phpType": "integer", - "pythonType": "int", - "rubyType": "Bignum or Fixnum (as required)" - }, - { - "protoType": "sfixed64", - "notes": "Always eight bytes.", - "cppType": "int64", - "csType": "long", - "goType": "int64", - "javaType": "long", - "phpType": "integer/string", - "pythonType": "int/long", - "rubyType": "Bignum" - }, - { - "protoType": "bool", - "notes": "", - "cppType": "bool", - "csType": "bool", - "goType": "bool", - "javaType": "boolean", - "phpType": "boolean", - "pythonType": "boolean", - "rubyType": "TrueClass/FalseClass" - }, - { - "protoType": "string", - "notes": "A string must always contain UTF-8 encoded or 7-bit ASCII text.", - "cppType": "string", - "csType": "string", - "goType": "string", - "javaType": "String", - "phpType": "string", - "pythonType": "str/unicode", - "rubyType": "String (UTF-8)" - }, - { - "protoType": "bytes", - "notes": "May contain any arbitrary sequence of bytes.", - "cppType": "string", - "csType": "ByteString", - "goType": "[]byte", - "javaType": "ByteString", - "phpType": "string", - "pythonType": "str", - "rubyType": "String (ASCII-8BIT)" - } - ] -} \ No newline at end of file diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts deleted file mode 100644 index e3a81ef11f..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts +++ /dev/null @@ -1,76 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/api_key.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { CreateAccountApiKeyRequest, CreateAccountApiKeyResponse, DeleteAccountApiKeyRequest, DeleteAccountApiKeyResponse, GetAccountApiKeyRequest, GetAccountApiKeyResponse, GetAccountApiKeysRequest, GetAccountApiKeysResponse, RegenerateAccountApiKeyRequest, RegenerateAccountApiKeyResponse } from "./api_key_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * Service that manages the lifecycle of API Keys that are associated with a specific Account. - * - * @generated from service mgmt.v1alpha1.ApiKeyService - */ -export const ApiKeyService = { - typeName: "mgmt.v1alpha1.ApiKeyService", - methods: { - /** - * Retrieves a list of Account API Keys - * - * @generated from rpc mgmt.v1alpha1.ApiKeyService.GetAccountApiKeys - */ - getAccountApiKeys: { - name: "GetAccountApiKeys", - I: GetAccountApiKeysRequest, - O: GetAccountApiKeysResponse, - kind: MethodKind.Unary, - }, - /** - * Retrieves a single API Key - * - * @generated from rpc mgmt.v1alpha1.ApiKeyService.GetAccountApiKey - */ - getAccountApiKey: { - name: "GetAccountApiKey", - I: GetAccountApiKeyRequest, - O: GetAccountApiKeyResponse, - kind: MethodKind.Unary, - }, - /** - * Creates a single API Key - * This method will return the decrypted contents of the API key - * - * @generated from rpc mgmt.v1alpha1.ApiKeyService.CreateAccountApiKey - */ - createAccountApiKey: { - name: "CreateAccountApiKey", - I: CreateAccountApiKeyRequest, - O: CreateAccountApiKeyResponse, - kind: MethodKind.Unary, - }, - /** - * Regenerates a single API Key with a new expiration time - * This method will return the decrypted contents of the API key - * - * @generated from rpc mgmt.v1alpha1.ApiKeyService.RegenerateAccountApiKey - */ - regenerateAccountApiKey: { - name: "RegenerateAccountApiKey", - I: RegenerateAccountApiKeyRequest, - O: RegenerateAccountApiKeyResponse, - kind: MethodKind.Unary, - }, - /** - * Deletes an API Key from the system. - * - * @generated from rpc mgmt.v1alpha1.ApiKeyService.DeleteAccountApiKey - */ - deleteAccountApiKey: { - name: "DeleteAccountApiKey", - I: DeleteAccountApiKeyRequest, - O: DeleteAccountApiKeyResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts deleted file mode 100644 index 0799c06d3d..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts +++ /dev/null @@ -1,491 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/api_key.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; - -/** - * @generated from message mgmt.v1alpha1.CreateAccountApiKeyRequest - */ -export class CreateAccountApiKeyRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * Validate between now and one year: now < x < 365 days - * - * @generated from field: google.protobuf.Timestamp expires_at = 3; - */ - expiresAt?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateAccountApiKeyRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "expires_at", kind: "message", T: Timestamp }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateAccountApiKeyRequest { - return new CreateAccountApiKeyRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateAccountApiKeyRequest { - return new CreateAccountApiKeyRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateAccountApiKeyRequest { - return new CreateAccountApiKeyRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateAccountApiKeyRequest | PlainMessage | undefined, b: CreateAccountApiKeyRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateAccountApiKeyRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateAccountApiKeyResponse - */ -export class CreateAccountApiKeyResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; - */ - apiKey?: AccountApiKey; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateAccountApiKeyResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateAccountApiKeyResponse { - return new CreateAccountApiKeyResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateAccountApiKeyResponse { - return new CreateAccountApiKeyResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateAccountApiKeyResponse { - return new CreateAccountApiKeyResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateAccountApiKeyResponse | PlainMessage | undefined, b: CreateAccountApiKeyResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateAccountApiKeyResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AccountApiKey - */ -export class AccountApiKey extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * The friendly name of the API Key - * - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: string account_id = 3; - */ - accountId = ""; - - /** - * @generated from field: string created_by_id = 4; - */ - createdById = ""; - - /** - * @generated from field: google.protobuf.Timestamp created_at = 5; - */ - createdAt?: Timestamp; - - /** - * @generated from field: string updated_by_id = 6; - */ - updatedById = ""; - - /** - * @generated from field: google.protobuf.Timestamp updated_at = 7; - */ - updatedAt?: Timestamp; - - /** - * key_value is only returned on initial creation or when it is regenerated - * - * @generated from field: optional string key_value = 8; - */ - keyValue?: string; - - /** - * @generated from field: string user_id = 9; - */ - userId = ""; - - /** - * The timestamp of what the API key expires and will not longer be usable. - * - * @generated from field: google.protobuf.Timestamp expires_at = 10; - */ - expiresAt?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccountApiKey"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "created_by_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "created_at", kind: "message", T: Timestamp }, - { no: 6, name: "updated_by_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 7, name: "updated_at", kind: "message", T: Timestamp }, - { no: 8, name: "key_value", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 9, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 10, name: "expires_at", kind: "message", T: Timestamp }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccountApiKey { - return new AccountApiKey().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccountApiKey { - return new AccountApiKey().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccountApiKey { - return new AccountApiKey().fromJsonString(jsonString, options); - } - - static equals(a: AccountApiKey | PlainMessage | undefined, b: AccountApiKey | PlainMessage | undefined): boolean { - return proto3.util.equals(AccountApiKey, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountApiKeysRequest - */ -export class GetAccountApiKeysRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeysRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeysRequest { - return new GetAccountApiKeysRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeysRequest { - return new GetAccountApiKeysRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeysRequest { - return new GetAccountApiKeysRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountApiKeysRequest | PlainMessage | undefined, b: GetAccountApiKeysRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountApiKeysRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountApiKeysResponse - */ -export class GetAccountApiKeysResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.AccountApiKey api_keys = 1; - */ - apiKeys: AccountApiKey[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeysResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "api_keys", kind: "message", T: AccountApiKey, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeysResponse { - return new GetAccountApiKeysResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeysResponse { - return new GetAccountApiKeysResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeysResponse { - return new GetAccountApiKeysResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountApiKeysResponse | PlainMessage | undefined, b: GetAccountApiKeysResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountApiKeysResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountApiKeyRequest - */ -export class GetAccountApiKeyRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeyRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeyRequest { - return new GetAccountApiKeyRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeyRequest { - return new GetAccountApiKeyRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeyRequest { - return new GetAccountApiKeyRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountApiKeyRequest | PlainMessage | undefined, b: GetAccountApiKeyRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountApiKeyRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountApiKeyResponse - */ -export class GetAccountApiKeyResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; - */ - apiKey?: AccountApiKey; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeyResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeyResponse { - return new GetAccountApiKeyResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeyResponse { - return new GetAccountApiKeyResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeyResponse { - return new GetAccountApiKeyResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountApiKeyResponse | PlainMessage | undefined, b: GetAccountApiKeyResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountApiKeyResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RegenerateAccountApiKeyRequest - */ -export class RegenerateAccountApiKeyRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * Validate between now and one year: now < x < 365 days - * - * @generated from field: google.protobuf.Timestamp expires_at = 2; - */ - expiresAt?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RegenerateAccountApiKeyRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "expires_at", kind: "message", T: Timestamp }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RegenerateAccountApiKeyRequest { - return new RegenerateAccountApiKeyRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RegenerateAccountApiKeyRequest { - return new RegenerateAccountApiKeyRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RegenerateAccountApiKeyRequest { - return new RegenerateAccountApiKeyRequest().fromJsonString(jsonString, options); - } - - static equals(a: RegenerateAccountApiKeyRequest | PlainMessage | undefined, b: RegenerateAccountApiKeyRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(RegenerateAccountApiKeyRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RegenerateAccountApiKeyResponse - */ -export class RegenerateAccountApiKeyResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; - */ - apiKey?: AccountApiKey; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RegenerateAccountApiKeyResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RegenerateAccountApiKeyResponse { - return new RegenerateAccountApiKeyResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RegenerateAccountApiKeyResponse { - return new RegenerateAccountApiKeyResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RegenerateAccountApiKeyResponse { - return new RegenerateAccountApiKeyResponse().fromJsonString(jsonString, options); - } - - static equals(a: RegenerateAccountApiKeyResponse | PlainMessage | undefined, b: RegenerateAccountApiKeyResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(RegenerateAccountApiKeyResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteAccountApiKeyRequest - */ -export class DeleteAccountApiKeyRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteAccountApiKeyRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteAccountApiKeyRequest { - return new DeleteAccountApiKeyRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteAccountApiKeyRequest { - return new DeleteAccountApiKeyRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteAccountApiKeyRequest { - return new DeleteAccountApiKeyRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteAccountApiKeyRequest | PlainMessage | undefined, b: DeleteAccountApiKeyRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteAccountApiKeyRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteAccountApiKeyResponse - */ -export class DeleteAccountApiKeyResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteAccountApiKeyResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteAccountApiKeyResponse { - return new DeleteAccountApiKeyResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteAccountApiKeyResponse { - return new DeleteAccountApiKeyResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteAccountApiKeyResponse { - return new DeleteAccountApiKeyResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteAccountApiKeyResponse | PlainMessage | undefined, b: DeleteAccountApiKeyResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteAccountApiKeyResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts deleted file mode 100644 index dcfdee3054..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts +++ /dev/null @@ -1,89 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/auth.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { CheckTokenRequest, CheckTokenResponse, GetAuthorizeUrlRequest, GetAuthorizeUrlResponse, GetAuthStatusRequest, GetAuthStatusResponse, GetCliIssuerRequest, GetCliIssuerResponse, LoginCliRequest, LoginCliResponse, RefreshCliRequest, RefreshCliResponse } from "./auth_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * Service that handles generic Authentication for Neosync - * Today this is mostly used by the CLI to receive authentication information - * - * @generated from service mgmt.v1alpha1.AuthService - */ -export const AuthService = { - typeName: "mgmt.v1alpha1.AuthService", - methods: { - /** - * Used by the CLI to login to Neosync with OAuth. - * - * @generated from rpc mgmt.v1alpha1.AuthService.LoginCli - */ - loginCli: { - name: "LoginCli", - I: LoginCliRequest, - O: LoginCliResponse, - kind: MethodKind.Unary, - }, - /** - * Used by the CLI to refresh an expired Neosync accesss token. - * This should only be used if an access token was previously retrieved from the `LoginCli` or `RefreshCli` methods. - * - * @generated from rpc mgmt.v1alpha1.AuthService.RefreshCli - */ - refreshCli: { - name: "RefreshCli", - I: RefreshCliRequest, - O: RefreshCliResponse, - kind: MethodKind.Unary, - }, - /** - * Empty endpoint to simply check if the provided access token is valid - * - * @generated from rpc mgmt.v1alpha1.AuthService.CheckToken - */ - checkToken: { - name: "CheckToken", - I: CheckTokenRequest, - O: CheckTokenResponse, - kind: MethodKind.Unary, - }, - /** - * Used by the CLI to retrieve Auth Issuer information - * @deprecated - * - * @generated from rpc mgmt.v1alpha1.AuthService.GetCliIssuer - */ - getCliIssuer: { - name: "GetCliIssuer", - I: GetCliIssuerRequest, - O: GetCliIssuerResponse, - kind: MethodKind.Unary, - }, - /** - * Used by the CLI to retrieve an Authorize URL for use with OAuth login. - * - * @generated from rpc mgmt.v1alpha1.AuthService.GetAuthorizeUrl - */ - getAuthorizeUrl: { - name: "GetAuthorizeUrl", - I: GetAuthorizeUrlRequest, - O: GetAuthorizeUrlResponse, - kind: MethodKind.Unary, - }, - /** - * Returns the auth status of the API server. Whether or not the backend has authentication enabled. - * This is used by clients to make decisions on whether or not they should send access tokens to the API. - * - * @generated from rpc mgmt.v1alpha1.AuthService.GetAuthStatus - */ - getAuthStatus: { - name: "GetAuthStatus", - I: GetAuthStatusRequest, - O: GetAuthStatusResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts deleted file mode 100644 index 0a69262c0c..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts +++ /dev/null @@ -1,559 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/auth.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, protoInt64 } from "@bufbuild/protobuf"; - -/** - * @generated from message mgmt.v1alpha1.LoginCliRequest - */ -export class LoginCliRequest extends Message { - /** - * The oauth code - * - * @generated from field: string code = 1; - */ - code = ""; - - /** - * The oauth redirect uri that the client uses during the oauth request - * - * @generated from field: string redirect_uri = 2; - */ - redirectUri = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.LoginCliRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "redirect_uri", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): LoginCliRequest { - return new LoginCliRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): LoginCliRequest { - return new LoginCliRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): LoginCliRequest { - return new LoginCliRequest().fromJsonString(jsonString, options); - } - - static equals(a: LoginCliRequest | PlainMessage | undefined, b: LoginCliRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(LoginCliRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.LoginCliResponse - */ -export class LoginCliResponse extends Message { - /** - * The access token that is returned on successful login - * - * @generated from field: mgmt.v1alpha1.AccessToken access_token = 1; - */ - accessToken?: AccessToken; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.LoginCliResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "access_token", kind: "message", T: AccessToken }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): LoginCliResponse { - return new LoginCliResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): LoginCliResponse { - return new LoginCliResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): LoginCliResponse { - return new LoginCliResponse().fromJsonString(jsonString, options); - } - - static equals(a: LoginCliResponse | PlainMessage | undefined, b: LoginCliResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(LoginCliResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAuthStatusRequest - */ -export class GetAuthStatusRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAuthStatusRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthStatusRequest { - return new GetAuthStatusRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthStatusRequest { - return new GetAuthStatusRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAuthStatusRequest { - return new GetAuthStatusRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAuthStatusRequest | PlainMessage | undefined, b: GetAuthStatusRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAuthStatusRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAuthStatusResponse - */ -export class GetAuthStatusResponse extends Message { - /** - * Whether or not the server has authentication enabled. - * This tells the client if it is expected to send access tokens. - * - * @generated from field: bool is_enabled = 1; - */ - isEnabled = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAuthStatusResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_enabled", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthStatusResponse { - return new GetAuthStatusResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthStatusResponse { - return new GetAuthStatusResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAuthStatusResponse { - return new GetAuthStatusResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAuthStatusResponse | PlainMessage | undefined, b: GetAuthStatusResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAuthStatusResponse, a, b); - } -} - -/** - * A decoded representation of an Access token from the backing auth server - * - * @generated from message mgmt.v1alpha1.AccessToken - */ -export class AccessToken extends Message { - /** - * The access token that will be provided in subsequent requests to provide authenticated access to the Api - * - * @generated from field: string access_token = 1; - */ - accessToken = ""; - - /** - * Token that can be used to retrieve a refreshed access token. - * Will not be provided if the offline_access scope is not provided in the initial login flow. - * - * @generated from field: optional string refresh_token = 2; - */ - refreshToken?: string; - - /** - * Relative time in seconds that the access token will expire. Combine with the current time to get the expires_at time. - * - * @generated from field: int64 expires_in = 3; - */ - expiresIn = protoInt64.zero; - - /** - * The scopes that the access token have - * - * @generated from field: string scope = 4; - */ - scope = ""; - - /** - * The identity token of the authenticated user - * - * @generated from field: optional string id_token = 5; - */ - idToken?: string; - - /** - * The token type. For JWTs, this will be `Bearer` - * - * @generated from field: string token_type = 6; - */ - tokenType = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccessToken"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "access_token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "refresh_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 3, name: "expires_in", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 4, name: "scope", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "id_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 6, name: "token_type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccessToken { - return new AccessToken().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccessToken { - return new AccessToken().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccessToken { - return new AccessToken().fromJsonString(jsonString, options); - } - - static equals(a: AccessToken | PlainMessage | undefined, b: AccessToken | PlainMessage | undefined): boolean { - return proto3.util.equals(AccessToken, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAuthorizeUrlRequest - */ -export class GetAuthorizeUrlRequest extends Message { - /** - * The state that's generated by the client that is passed along to prevent tampering - * - * @generated from field: string state = 1; - */ - state = ""; - - /** - * The redirect uri that the client will be redirected back to during the auth request - * - * @generated from field: string redirect_uri = 2; - */ - redirectUri = ""; - - /** - * The scopes the client is requesting as a part of the oauth login request - * - * @generated from field: string scope = 3; - */ - scope = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAuthorizeUrlRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "state", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "redirect_uri", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "scope", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthorizeUrlRequest { - return new GetAuthorizeUrlRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthorizeUrlRequest { - return new GetAuthorizeUrlRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAuthorizeUrlRequest { - return new GetAuthorizeUrlRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAuthorizeUrlRequest | PlainMessage | undefined, b: GetAuthorizeUrlRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAuthorizeUrlRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAuthorizeUrlResponse - */ -export class GetAuthorizeUrlResponse extends Message { - /** - * The generated url that is the client will be redirected to during the Oauth flow - * - * @generated from field: string url = 1; - */ - url = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAuthorizeUrlResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthorizeUrlResponse { - return new GetAuthorizeUrlResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthorizeUrlResponse { - return new GetAuthorizeUrlResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAuthorizeUrlResponse { - return new GetAuthorizeUrlResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAuthorizeUrlResponse | PlainMessage | undefined, b: GetAuthorizeUrlResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAuthorizeUrlResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetCliIssuerRequest - */ -export class GetCliIssuerRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetCliIssuerRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetCliIssuerRequest { - return new GetCliIssuerRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetCliIssuerRequest { - return new GetCliIssuerRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetCliIssuerRequest { - return new GetCliIssuerRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetCliIssuerRequest | PlainMessage | undefined, b: GetCliIssuerRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetCliIssuerRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetCliIssuerResponse - */ -export class GetCliIssuerResponse extends Message { - /** - * The backing authentication issuer url - * - * @generated from field: string issuer_url = 1; - */ - issuerUrl = ""; - - /** - * The audience that will be used in the access token. This corresponds to the "aud" claim - * - * @generated from field: string audience = 2; - */ - audience = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetCliIssuerResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "issuer_url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "audience", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetCliIssuerResponse { - return new GetCliIssuerResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetCliIssuerResponse { - return new GetCliIssuerResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetCliIssuerResponse { - return new GetCliIssuerResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetCliIssuerResponse | PlainMessage | undefined, b: GetCliIssuerResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetCliIssuerResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RefreshCliRequest - */ -export class RefreshCliRequest extends Message { - /** - * The token used to retrieve a new access token. - * - * @generated from field: string refresh_token = 1; - */ - refreshToken = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RefreshCliRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "refresh_token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RefreshCliRequest { - return new RefreshCliRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RefreshCliRequest { - return new RefreshCliRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RefreshCliRequest { - return new RefreshCliRequest().fromJsonString(jsonString, options); - } - - static equals(a: RefreshCliRequest | PlainMessage | undefined, b: RefreshCliRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(RefreshCliRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RefreshCliResponse - */ -export class RefreshCliResponse extends Message { - /** - * The access token that is returned on successful refresh - * - * @generated from field: mgmt.v1alpha1.AccessToken access_token = 1; - */ - accessToken?: AccessToken; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RefreshCliResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "access_token", kind: "message", T: AccessToken }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RefreshCliResponse { - return new RefreshCliResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RefreshCliResponse { - return new RefreshCliResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RefreshCliResponse { - return new RefreshCliResponse().fromJsonString(jsonString, options); - } - - static equals(a: RefreshCliResponse | PlainMessage | undefined, b: RefreshCliResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(RefreshCliResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckTokenRequest - */ -export class CheckTokenRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckTokenRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckTokenRequest { - return new CheckTokenRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckTokenRequest { - return new CheckTokenRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckTokenRequest { - return new CheckTokenRequest().fromJsonString(jsonString, options); - } - - static equals(a: CheckTokenRequest | PlainMessage | undefined, b: CheckTokenRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckTokenRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckTokenResponse - */ -export class CheckTokenResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckTokenResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckTokenResponse { - return new CheckTokenResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckTokenResponse { - return new CheckTokenResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckTokenResponse { - return new CheckTokenResponse().fromJsonString(jsonString, options); - } - - static equals(a: CheckTokenResponse | PlainMessage | undefined, b: CheckTokenResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckTokenResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts deleted file mode 100644 index 3d3720b51a..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts +++ /dev/null @@ -1,110 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/connection.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { CheckConnectionConfigRequest, CheckConnectionConfigResponse, CheckSqlQueryRequest, CheckSqlQueryResponse, CreateConnectionRequest, CreateConnectionResponse, DeleteConnectionRequest, DeleteConnectionResponse, GetConnectionRequest, GetConnectionResponse, GetConnectionsRequest, GetConnectionsResponse, IsConnectionNameAvailableRequest, IsConnectionNameAvailableResponse, UpdateConnectionRequest, UpdateConnectionResponse } from "./connection_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * Service for managing datasource connections. - * This is a primary data model in Neosync and is used in reference when hooking up Jobs to synchronize and generate data. - * - * @generated from service mgmt.v1alpha1.ConnectionService - */ -export const ConnectionService = { - typeName: "mgmt.v1alpha1.ConnectionService", - methods: { - /** - * Returns a list of connections associated with the account - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.GetConnections - */ - getConnections: { - name: "GetConnections", - I: GetConnectionsRequest, - O: GetConnectionsResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a single connection - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.GetConnection - */ - getConnection: { - name: "GetConnection", - I: GetConnectionRequest, - O: GetConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * Creates a new connection - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.CreateConnection - */ - createConnection: { - name: "CreateConnection", - I: CreateConnectionRequest, - O: CreateConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * Updates an existing connection - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.UpdateConnection - */ - updateConnection: { - name: "UpdateConnection", - I: UpdateConnectionRequest, - O: UpdateConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * Removes a connection from the system. - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.DeleteConnection - */ - deleteConnection: { - name: "DeleteConnection", - I: DeleteConnectionRequest, - O: DeleteConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * Connections have friendly names, this method checks if the requested name is available in the system based on the account - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.IsConnectionNameAvailable - */ - isConnectionNameAvailable: { - name: "IsConnectionNameAvailable", - I: IsConnectionNameAvailableRequest, - O: IsConnectionNameAvailableResponse, - kind: MethodKind.Unary, - }, - /** - * Checks if the connection config is connectable by the backend. - * Used mostly to verify that a connection is valid prior to creating a Connection object. - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.CheckConnectionConfig - */ - checkConnectionConfig: { - name: "CheckConnectionConfig", - I: CheckConnectionConfigRequest, - O: CheckConnectionConfigResponse, - kind: MethodKind.Unary, - }, - /** - * Checks a constructed SQL query against a sql-based connection to see if it's valid based on that connection's data schema - * This is useful when constructing subsets to see if the WHERE clause is correct - * - * @generated from rpc mgmt.v1alpha1.ConnectionService.CheckSqlQuery - */ - checkSqlQuery: { - name: "CheckSqlQuery", - I: CheckSqlQueryRequest, - O: CheckSqlQueryResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts deleted file mode 100644 index e4cd5329e6..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts +++ /dev/null @@ -1,90 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/connection_data.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { GetConnectionDataStreamRequest, GetConnectionDataStreamResponse, GetConnectionForeignConstraintsRequest, GetConnectionForeignConstraintsResponse, GetConnectionInitStatementsRequest, GetConnectionInitStatementsResponse, GetConnectionPrimaryConstraintsRequest, GetConnectionPrimaryConstraintsResponse, GetConnectionSchemaRequest, GetConnectionSchemaResponse, GetConnectionUniqueConstraintsRequest, GetConnectionUniqueConstraintsResponse } from "./connection_data_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * Service for managing connection data. - * This is used in handle data from a connection - * - * @generated from service mgmt.v1alpha1.ConnectionDataService - */ -export const ConnectionDataService = { - typeName: "mgmt.v1alpha1.ConnectionDataService", - methods: { - /** - * Streaming endpoint that will stream the data available from the Connection to the client. - * Used primarily by the CLI sync command. - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionDataStream - */ - getConnectionDataStream: { - name: "GetConnectionDataStream", - I: GetConnectionDataStreamRequest, - O: GetConnectionDataStreamResponse, - kind: MethodKind.ServerStreaming, - }, - /** - * Returns the schema for a specific connection. Used mostly for SQL-based connections - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionSchema - */ - getConnectionSchema: { - name: "GetConnectionSchema", - I: GetConnectionSchemaRequest, - O: GetConnectionSchemaResponse, - kind: MethodKind.Unary, - }, - /** - * For a specific connection, returns the foreign key constraints. Mostly useful for SQL-based Connections. - * Used primarily by the CLI sync command to determine stream order. - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionForeignConstraints - */ - getConnectionForeignConstraints: { - name: "GetConnectionForeignConstraints", - I: GetConnectionForeignConstraintsRequest, - O: GetConnectionForeignConstraintsResponse, - kind: MethodKind.Unary, - }, - /** - * For a specific connection, returns the primary key constraints. Mostly useful for SQL-based Connections. - * Used primarily by the CLI sync command to determine stream order. - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionPrimaryConstraints - */ - getConnectionPrimaryConstraints: { - name: "GetConnectionPrimaryConstraints", - I: GetConnectionPrimaryConstraintsRequest, - O: GetConnectionPrimaryConstraintsResponse, - kind: MethodKind.Unary, - }, - /** - * For a specific connection, returns the init table statements. Mostly useful for SQL-based Connections. - * Used primarily by the CLI sync command to create table schema init statement. - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionInitStatements - */ - getConnectionInitStatements: { - name: "GetConnectionInitStatements", - I: GetConnectionInitStatementsRequest, - O: GetConnectionInitStatementsResponse, - kind: MethodKind.Unary, - }, - /** - * For a specific connection, returns the unique constraints. Mostly useful for SQL-based connections. - * - * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionUniqueConstraints - */ - getConnectionUniqueConstraints: { - name: "GetConnectionUniqueConstraints", - I: GetConnectionUniqueConstraintsRequest, - O: GetConnectionUniqueConstraintsResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts deleted file mode 100644 index 142806ac5d..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts +++ /dev/null @@ -1,1171 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/connection_data.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3 } from "@bufbuild/protobuf"; - -/** - * @generated from message mgmt.v1alpha1.PostgresStreamConfig - */ -export class PostgresStreamConfig extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresStreamConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresStreamConfig { - return new PostgresStreamConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresStreamConfig { - return new PostgresStreamConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresStreamConfig { - return new PostgresStreamConfig().fromJsonString(jsonString, options); - } - - static equals(a: PostgresStreamConfig | PlainMessage | undefined, b: PostgresStreamConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresStreamConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlStreamConfig - */ -export class MysqlStreamConfig extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlStreamConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlStreamConfig { - return new MysqlStreamConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlStreamConfig { - return new MysqlStreamConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlStreamConfig { - return new MysqlStreamConfig().fromJsonString(jsonString, options); - } - - static equals(a: MysqlStreamConfig | PlainMessage | undefined, b: MysqlStreamConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlStreamConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AwsS3StreamConfig - */ -export class AwsS3StreamConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.AwsS3StreamConfig.id - */ - id: { - /** - * @generated from field: string job_id = 1; - */ - value: string; - case: "jobId"; - } | { - /** - * @generated from field: string job_run_id = 2; - */ - value: string; - case: "jobRunId"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3StreamConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3StreamConfig { - return new AwsS3StreamConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3StreamConfig { - return new AwsS3StreamConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3StreamConfig { - return new AwsS3StreamConfig().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3StreamConfig | PlainMessage | undefined, b: AwsS3StreamConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3StreamConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConnectionStreamConfig - */ -export class ConnectionStreamConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.ConnectionStreamConfig.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.PostgresStreamConfig pg_config = 1; - */ - value: PostgresStreamConfig; - case: "pgConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.AwsS3StreamConfig aws_s3_config = 2; - */ - value: AwsS3StreamConfig; - case: "awsS3Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlStreamConfig mysql_config = 3; - */ - value: MysqlStreamConfig; - case: "mysqlConfig"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConnectionStreamConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "pg_config", kind: "message", T: PostgresStreamConfig, oneof: "config" }, - { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3StreamConfig, oneof: "config" }, - { no: 3, name: "mysql_config", kind: "message", T: MysqlStreamConfig, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionStreamConfig { - return new ConnectionStreamConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionStreamConfig { - return new ConnectionStreamConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConnectionStreamConfig { - return new ConnectionStreamConfig().fromJsonString(jsonString, options); - } - - static equals(a: ConnectionStreamConfig | PlainMessage | undefined, b: ConnectionStreamConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(ConnectionStreamConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionDataStreamRequest - */ -export class GetConnectionDataStreamRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.ConnectionStreamConfig stream_config = 2; - */ - streamConfig?: ConnectionStreamConfig; - - /** - * @generated from field: string schema = 3; - */ - schema = ""; - - /** - * @generated from field: string table = 4; - */ - table = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionDataStreamRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "stream_config", kind: "message", T: ConnectionStreamConfig }, - { no: 3, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionDataStreamRequest { - return new GetConnectionDataStreamRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionDataStreamRequest { - return new GetConnectionDataStreamRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionDataStreamRequest { - return new GetConnectionDataStreamRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionDataStreamRequest | PlainMessage | undefined, b: GetConnectionDataStreamRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionDataStreamRequest, a, b); - } -} - -/** - * Each stream response is a single row in the requested schema and table - * - * @generated from message mgmt.v1alpha1.GetConnectionDataStreamResponse - */ -export class GetConnectionDataStreamResponse extends Message { - /** - * A map of column name to the bytes value of the data that was found for that column and row - * - * @generated from field: map row = 1; - */ - row: { [key: string]: Uint8Array } = {}; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionDataStreamResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "row", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 12 /* ScalarType.BYTES */} }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionDataStreamResponse { - return new GetConnectionDataStreamResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionDataStreamResponse { - return new GetConnectionDataStreamResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionDataStreamResponse { - return new GetConnectionDataStreamResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionDataStreamResponse | PlainMessage | undefined, b: GetConnectionDataStreamResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionDataStreamResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresSchemaConfig - */ -export class PostgresSchemaConfig extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresSchemaConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSchemaConfig { - return new PostgresSchemaConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSchemaConfig { - return new PostgresSchemaConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresSchemaConfig { - return new PostgresSchemaConfig().fromJsonString(jsonString, options); - } - - static equals(a: PostgresSchemaConfig | PlainMessage | undefined, b: PostgresSchemaConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresSchemaConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlSchemaConfig - */ -export class MysqlSchemaConfig extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlSchemaConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSchemaConfig { - return new MysqlSchemaConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSchemaConfig { - return new MysqlSchemaConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlSchemaConfig { - return new MysqlSchemaConfig().fromJsonString(jsonString, options); - } - - static equals(a: MysqlSchemaConfig | PlainMessage | undefined, b: MysqlSchemaConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlSchemaConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AwsS3SchemaConfig - */ -export class AwsS3SchemaConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.AwsS3SchemaConfig.id - */ - id: { - /** - * @generated from field: string job_id = 1; - */ - value: string; - case: "jobId"; - } | { - /** - * @generated from field: string job_run_id = 2; - */ - value: string; - case: "jobRunId"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3SchemaConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3SchemaConfig { - return new AwsS3SchemaConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3SchemaConfig { - return new AwsS3SchemaConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3SchemaConfig { - return new AwsS3SchemaConfig().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3SchemaConfig | PlainMessage | undefined, b: AwsS3SchemaConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3SchemaConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConnectionSchemaConfig - */ -export class ConnectionSchemaConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.ConnectionSchemaConfig.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.PostgresSchemaConfig pg_config = 1; - */ - value: PostgresSchemaConfig; - case: "pgConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.AwsS3SchemaConfig aws_s3_config = 2; - */ - value: AwsS3SchemaConfig; - case: "awsS3Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlSchemaConfig mysql_config = 3; - */ - value: MysqlSchemaConfig; - case: "mysqlConfig"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConnectionSchemaConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "pg_config", kind: "message", T: PostgresSchemaConfig, oneof: "config" }, - { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3SchemaConfig, oneof: "config" }, - { no: 3, name: "mysql_config", kind: "message", T: MysqlSchemaConfig, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionSchemaConfig { - return new ConnectionSchemaConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionSchemaConfig { - return new ConnectionSchemaConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConnectionSchemaConfig { - return new ConnectionSchemaConfig().fromJsonString(jsonString, options); - } - - static equals(a: ConnectionSchemaConfig | PlainMessage | undefined, b: ConnectionSchemaConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(ConnectionSchemaConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DatabaseColumn - */ -export class DatabaseColumn extends Message { - /** - * The database schema. Ex: public - * - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * The name of the table in the schema - * - * @generated from field: string table = 2; - */ - table = ""; - - /** - * The name of the column - * - * @generated from field: string column = 3; - */ - column = ""; - - /** - * The datatype of the column - * - * @generated from field: string data_type = 4; - */ - dataType = ""; - - /** - * The isNullable Flag of the column - * - * @generated from field: string is_nullable = 5; - */ - isNullable = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DatabaseColumn"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "data_type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "is_nullable", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DatabaseColumn { - return new DatabaseColumn().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DatabaseColumn { - return new DatabaseColumn().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DatabaseColumn { - return new DatabaseColumn().fromJsonString(jsonString, options); - } - - static equals(a: DatabaseColumn | PlainMessage | undefined, b: DatabaseColumn | PlainMessage | undefined): boolean { - return proto3.util.equals(DatabaseColumn, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionSchemaRequest - */ -export class GetConnectionSchemaRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.ConnectionSchemaConfig schema_config = 2; - */ - schemaConfig?: ConnectionSchemaConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionSchemaRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "schema_config", kind: "message", T: ConnectionSchemaConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionSchemaRequest { - return new GetConnectionSchemaRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionSchemaRequest { - return new GetConnectionSchemaRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionSchemaRequest { - return new GetConnectionSchemaRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionSchemaRequest | PlainMessage | undefined, b: GetConnectionSchemaRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionSchemaRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionSchemaResponse - */ -export class GetConnectionSchemaResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.DatabaseColumn schemas = 1; - */ - schemas: DatabaseColumn[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionSchemaResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schemas", kind: "message", T: DatabaseColumn, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionSchemaResponse { - return new GetConnectionSchemaResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionSchemaResponse { - return new GetConnectionSchemaResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionSchemaResponse { - return new GetConnectionSchemaResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionSchemaResponse | PlainMessage | undefined, b: GetConnectionSchemaResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionSchemaResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionForeignConstraintsRequest - */ -export class GetConnectionForeignConstraintsRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionForeignConstraintsRequest { - return new GetConnectionForeignConstraintsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionForeignConstraintsRequest { - return new GetConnectionForeignConstraintsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionForeignConstraintsRequest { - return new GetConnectionForeignConstraintsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionForeignConstraintsRequest | PlainMessage | undefined, b: GetConnectionForeignConstraintsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionForeignConstraintsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ForeignKey - */ -export class ForeignKey extends Message { - /** - * @generated from field: string table = 1; - */ - table = ""; - - /** - * @generated from field: string column = 2; - */ - column = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ForeignKey"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ForeignKey { - return new ForeignKey().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ForeignKey { - return new ForeignKey().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ForeignKey { - return new ForeignKey().fromJsonString(jsonString, options); - } - - static equals(a: ForeignKey | PlainMessage | undefined, b: ForeignKey | PlainMessage | undefined): boolean { - return proto3.util.equals(ForeignKey, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ForeignConstraint - */ -export class ForeignConstraint extends Message { - /** - * @generated from field: string column = 1; - */ - column = ""; - - /** - * @generated from field: bool is_nullable = 2; - */ - isNullable = false; - - /** - * @generated from field: mgmt.v1alpha1.ForeignKey foreign_key = 3; - */ - foreignKey?: ForeignKey; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ForeignConstraint"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "is_nullable", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "foreign_key", kind: "message", T: ForeignKey }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ForeignConstraint { - return new ForeignConstraint().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ForeignConstraint { - return new ForeignConstraint().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ForeignConstraint { - return new ForeignConstraint().fromJsonString(jsonString, options); - } - - static equals(a: ForeignConstraint | PlainMessage | undefined, b: ForeignConstraint | PlainMessage | undefined): boolean { - return proto3.util.equals(ForeignConstraint, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ForeignConstraintTables - */ -export class ForeignConstraintTables extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.ForeignConstraint constraints = 1; - */ - constraints: ForeignConstraint[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ForeignConstraintTables"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "constraints", kind: "message", T: ForeignConstraint, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ForeignConstraintTables { - return new ForeignConstraintTables().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ForeignConstraintTables { - return new ForeignConstraintTables().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ForeignConstraintTables { - return new ForeignConstraintTables().fromJsonString(jsonString, options); - } - - static equals(a: ForeignConstraintTables | PlainMessage | undefined, b: ForeignConstraintTables | PlainMessage | undefined): boolean { - return proto3.util.equals(ForeignConstraintTables, a, b); - } -} - -/** - * Dependency constraints for a specific table - * - * @generated from message mgmt.v1alpha1.GetConnectionForeignConstraintsResponse - */ -export class GetConnectionForeignConstraintsResponse extends Message { - /** - * the key here is . and the list of tables that it depends on, also `.
` format. - * - * @generated from field: map table_constraints = 1; - */ - tableConstraints: { [key: string]: ForeignConstraintTables } = {}; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: ForeignConstraintTables} }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionForeignConstraintsResponse { - return new GetConnectionForeignConstraintsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionForeignConstraintsResponse { - return new GetConnectionForeignConstraintsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionForeignConstraintsResponse { - return new GetConnectionForeignConstraintsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionForeignConstraintsResponse | PlainMessage | undefined, b: GetConnectionForeignConstraintsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionForeignConstraintsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.InitStatementOptions - */ -export class InitStatementOptions extends Message { - /** - * @generated from field: bool init_schema = 1; - */ - initSchema = false; - - /** - * @generated from field: bool truncate_before_insert = 2; - */ - truncateBeforeInsert = false; - - /** - * @generated from field: bool truncate_cascade = 3; - */ - truncateCascade = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.InitStatementOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "init_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "truncate_cascade", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): InitStatementOptions { - return new InitStatementOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): InitStatementOptions { - return new InitStatementOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): InitStatementOptions { - return new InitStatementOptions().fromJsonString(jsonString, options); - } - - static equals(a: InitStatementOptions | PlainMessage | undefined, b: InitStatementOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(InitStatementOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionInitStatementsRequest - */ -export class GetConnectionInitStatementsRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.InitStatementOptions options = 2; - */ - options?: InitStatementOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionInitStatementsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "options", kind: "message", T: InitStatementOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionInitStatementsRequest { - return new GetConnectionInitStatementsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionInitStatementsRequest { - return new GetConnectionInitStatementsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionInitStatementsRequest { - return new GetConnectionInitStatementsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionInitStatementsRequest | PlainMessage | undefined, b: GetConnectionInitStatementsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionInitStatementsRequest, a, b); - } -} - -/** - * Init statement for a specific table - * - * @generated from message mgmt.v1alpha1.GetConnectionInitStatementsResponse - */ -export class GetConnectionInitStatementsResponse extends Message { - /** - * the key here is .
and value is the table init statement. - * - * @generated from field: map table_init_statements = 1; - */ - tableInitStatements: { [key: string]: string } = {}; - - /** - * the key here is .
and value is the table truncate statement. - * - * @generated from field: map table_truncate_statements = 2; - */ - tableTruncateStatements: { [key: string]: string } = {}; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionInitStatementsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table_init_statements", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 9 /* ScalarType.STRING */} }, - { no: 2, name: "table_truncate_statements", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 9 /* ScalarType.STRING */} }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionInitStatementsResponse { - return new GetConnectionInitStatementsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionInitStatementsResponse { - return new GetConnectionInitStatementsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionInitStatementsResponse { - return new GetConnectionInitStatementsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionInitStatementsResponse | PlainMessage | undefined, b: GetConnectionInitStatementsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionInitStatementsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PrimaryConstraint - */ -export class PrimaryConstraint extends Message { - /** - * @generated from field: repeated string columns = 1; - */ - columns: string[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PrimaryConstraint"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "columns", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PrimaryConstraint { - return new PrimaryConstraint().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PrimaryConstraint { - return new PrimaryConstraint().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PrimaryConstraint { - return new PrimaryConstraint().fromJsonString(jsonString, options); - } - - static equals(a: PrimaryConstraint | PlainMessage | undefined, b: PrimaryConstraint | PlainMessage | undefined): boolean { - return proto3.util.equals(PrimaryConstraint, a, b); - } -} - -/** - * Primary constraints for a specific table - * - * @generated from message mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest - */ -export class GetConnectionPrimaryConstraintsRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionPrimaryConstraintsRequest { - return new GetConnectionPrimaryConstraintsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionPrimaryConstraintsRequest { - return new GetConnectionPrimaryConstraintsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionPrimaryConstraintsRequest { - return new GetConnectionPrimaryConstraintsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionPrimaryConstraintsRequest | PlainMessage | undefined, b: GetConnectionPrimaryConstraintsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionPrimaryConstraintsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse - */ -export class GetConnectionPrimaryConstraintsResponse extends Message { - /** - * the key here is .
and value is the primary constraint - * - * @generated from field: map table_constraints = 1; - */ - tableConstraints: { [key: string]: PrimaryConstraint } = {}; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: PrimaryConstraint} }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionPrimaryConstraintsResponse { - return new GetConnectionPrimaryConstraintsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionPrimaryConstraintsResponse { - return new GetConnectionPrimaryConstraintsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionPrimaryConstraintsResponse { - return new GetConnectionPrimaryConstraintsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionPrimaryConstraintsResponse | PlainMessage | undefined, b: GetConnectionPrimaryConstraintsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionPrimaryConstraintsResponse, a, b); - } -} - -/** - * Unique constraints for a specific table - * - * @generated from message mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest - */ -export class GetConnectionUniqueConstraintsRequest extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionUniqueConstraintsRequest { - return new GetConnectionUniqueConstraintsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionUniqueConstraintsRequest { - return new GetConnectionUniqueConstraintsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionUniqueConstraintsRequest { - return new GetConnectionUniqueConstraintsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionUniqueConstraintsRequest | PlainMessage | undefined, b: GetConnectionUniqueConstraintsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionUniqueConstraintsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse - */ -export class GetConnectionUniqueConstraintsResponse extends Message { - /** - * the key here is .
and value is the unique constraint - * - * @generated from field: map table_constraints = 1; - */ - tableConstraints: { [key: string]: UniqueConstraint } = {}; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: UniqueConstraint} }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionUniqueConstraintsResponse { - return new GetConnectionUniqueConstraintsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionUniqueConstraintsResponse { - return new GetConnectionUniqueConstraintsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionUniqueConstraintsResponse { - return new GetConnectionUniqueConstraintsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionUniqueConstraintsResponse | PlainMessage | undefined, b: GetConnectionUniqueConstraintsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionUniqueConstraintsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UniqueConstraint - */ -export class UniqueConstraint extends Message { - /** - * @generated from field: repeated string columns = 1; - */ - columns: string[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UniqueConstraint"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "columns", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UniqueConstraint { - return new UniqueConstraint().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UniqueConstraint { - return new UniqueConstraint().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UniqueConstraint { - return new UniqueConstraint().fromJsonString(jsonString, options); - } - - static equals(a: UniqueConstraint | PlainMessage | undefined, b: UniqueConstraint | PlainMessage | undefined): boolean { - return proto3.util.equals(UniqueConstraint, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts deleted file mode 100644 index 53530d947c..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts +++ /dev/null @@ -1,1575 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/connection.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; - -/** - * @generated from message mgmt.v1alpha1.GetConnectionsRequest - */ -export class GetConnectionsRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionsRequest { - return new GetConnectionsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionsRequest { - return new GetConnectionsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionsRequest { - return new GetConnectionsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionsRequest | PlainMessage | undefined, b: GetConnectionsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionsResponse - */ -export class GetConnectionsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.Connection connections = 1; - */ - connections: Connection[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connections", kind: "message", T: Connection, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionsResponse { - return new GetConnectionsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionsResponse { - return new GetConnectionsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionsResponse { - return new GetConnectionsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionsResponse | PlainMessage | undefined, b: GetConnectionsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionRequest - */ -export class GetConnectionRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionRequest { - return new GetConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionRequest { - return new GetConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionRequest { - return new GetConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionRequest | PlainMessage | undefined, b: GetConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetConnectionResponse - */ -export class GetConnectionResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Connection connection = 1; - */ - connection?: Connection; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection", kind: "message", T: Connection }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionResponse { - return new GetConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionResponse { - return new GetConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetConnectionResponse { - return new GetConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetConnectionResponse | PlainMessage | undefined, b: GetConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateConnectionRequest - */ -export class CreateConnectionRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * The friendly name of the connection - * - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; - */ - connectionConfig?: ConnectionConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateConnectionRequest { - return new CreateConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateConnectionRequest { - return new CreateConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateConnectionRequest { - return new CreateConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateConnectionRequest | PlainMessage | undefined, b: CreateConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateConnectionResponse - */ -export class CreateConnectionResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Connection connection = 1; - */ - connection?: Connection; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection", kind: "message", T: Connection }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateConnectionResponse { - return new CreateConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateConnectionResponse { - return new CreateConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateConnectionResponse { - return new CreateConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateConnectionResponse | PlainMessage | undefined, b: CreateConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateConnectionRequest - */ -export class UpdateConnectionRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; - */ - connectionConfig?: ConnectionConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateConnectionRequest { - return new UpdateConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateConnectionRequest { - return new UpdateConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateConnectionRequest { - return new UpdateConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: UpdateConnectionRequest | PlainMessage | undefined, b: UpdateConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateConnectionResponse - */ -export class UpdateConnectionResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Connection connection = 1; - */ - connection?: Connection; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection", kind: "message", T: Connection }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateConnectionResponse { - return new UpdateConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateConnectionResponse { - return new UpdateConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateConnectionResponse { - return new UpdateConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: UpdateConnectionResponse | PlainMessage | undefined, b: UpdateConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteConnectionRequest - */ -export class DeleteConnectionRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteConnectionRequest { - return new DeleteConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteConnectionRequest { - return new DeleteConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteConnectionRequest { - return new DeleteConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteConnectionRequest | PlainMessage | undefined, b: DeleteConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteConnectionResponse - */ -export class DeleteConnectionResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteConnectionResponse { - return new DeleteConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteConnectionResponse { - return new DeleteConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteConnectionResponse { - return new DeleteConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteConnectionResponse | PlainMessage | undefined, b: DeleteConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckConnectionConfigRequest - */ -export class CheckConnectionConfigRequest extends Message { - /** - * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 1; - */ - connectionConfig?: ConnectionConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckConnectionConfigRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_config", kind: "message", T: ConnectionConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckConnectionConfigRequest { - return new CheckConnectionConfigRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckConnectionConfigRequest { - return new CheckConnectionConfigRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckConnectionConfigRequest { - return new CheckConnectionConfigRequest().fromJsonString(jsonString, options); - } - - static equals(a: CheckConnectionConfigRequest | PlainMessage | undefined, b: CheckConnectionConfigRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckConnectionConfigRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckConnectionConfigResponse - */ -export class CheckConnectionConfigResponse extends Message { - /** - * Whether or not the API was able to ping the connection - * - * @generated from field: bool is_connected = 1; - */ - isConnected = false; - - /** - * This is the error that was received if the API was unable to connect - * - * @generated from field: optional string connection_error = 2; - */ - connectionError?: string; - - /** - * @generated from field: repeated mgmt.v1alpha1.ConnectionRolePrivilege privileges = 3; - */ - privileges: ConnectionRolePrivilege[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckConnectionConfigResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_connected", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "connection_error", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 3, name: "privileges", kind: "message", T: ConnectionRolePrivilege, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckConnectionConfigResponse { - return new CheckConnectionConfigResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckConnectionConfigResponse { - return new CheckConnectionConfigResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckConnectionConfigResponse { - return new CheckConnectionConfigResponse().fromJsonString(jsonString, options); - } - - static equals(a: CheckConnectionConfigResponse | PlainMessage | undefined, b: CheckConnectionConfigResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckConnectionConfigResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConnectionRolePrivilege - */ -export class ConnectionRolePrivilege extends Message { - /** - * The role that was given the permissions - * - * @generated from field: string grantee = 1; - */ - grantee = ""; - - /** - * The database schema. Ex: public - * - * @generated from field: string schema = 2; - */ - schema = ""; - - /** - * The name of the table in the schema - * - * @generated from field: string table = 3; - */ - table = ""; - - /** - * The privileges given to that role - * - * @generated from field: repeated string privilege_type = 4; - */ - privilegeType: string[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConnectionRolePrivilege"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "grantee", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "privilege_type", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionRolePrivilege { - return new ConnectionRolePrivilege().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionRolePrivilege { - return new ConnectionRolePrivilege().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConnectionRolePrivilege { - return new ConnectionRolePrivilege().fromJsonString(jsonString, options); - } - - static equals(a: ConnectionRolePrivilege | PlainMessage | undefined, b: ConnectionRolePrivilege | PlainMessage | undefined): boolean { - return proto3.util.equals(ConnectionRolePrivilege, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.Connection - */ -export class Connection extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; - */ - connectionConfig?: ConnectionConfig; - - /** - * @generated from field: string created_by_user_id = 4; - */ - createdByUserId = ""; - - /** - * @generated from field: google.protobuf.Timestamp created_at = 5; - */ - createdAt?: Timestamp; - - /** - * @generated from field: string updated_by_user_id = 6; - */ - updatedByUserId = ""; - - /** - * @generated from field: google.protobuf.Timestamp updated_at = 7; - */ - updatedAt?: Timestamp; - - /** - * @generated from field: string account_id = 8; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.Connection"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, - { no: 4, name: "created_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "created_at", kind: "message", T: Timestamp }, - { no: 6, name: "updated_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 7, name: "updated_at", kind: "message", T: Timestamp }, - { no: 8, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): Connection { - return new Connection().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): Connection { - return new Connection().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): Connection { - return new Connection().fromJsonString(jsonString, options); - } - - static equals(a: Connection | PlainMessage | undefined, b: Connection | PlainMessage | undefined): boolean { - return proto3.util.equals(Connection, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConnectionConfig - */ -export class ConnectionConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.ConnectionConfig.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.PostgresConnectionConfig pg_config = 1; - */ - value: PostgresConnectionConfig; - case: "pgConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.AwsS3ConnectionConfig aws_s3_config = 2; - */ - value: AwsS3ConnectionConfig; - case: "awsS3Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlConnectionConfig mysql_config = 3; - */ - value: MysqlConnectionConfig; - case: "mysqlConfig"; - } | { - /** - * Configures a connection to a directory available on the local file system - * - * @generated from field: mgmt.v1alpha1.LocalDirectoryConnectionConfig local_dir_config = 4; - */ - value: LocalDirectoryConnectionConfig; - case: "localDirConfig"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConnectionConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "pg_config", kind: "message", T: PostgresConnectionConfig, oneof: "config" }, - { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3ConnectionConfig, oneof: "config" }, - { no: 3, name: "mysql_config", kind: "message", T: MysqlConnectionConfig, oneof: "config" }, - { no: 4, name: "local_dir_config", kind: "message", T: LocalDirectoryConnectionConfig, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionConfig { - return new ConnectionConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionConfig { - return new ConnectionConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConnectionConfig { - return new ConnectionConfig().fromJsonString(jsonString, options); - } - - static equals(a: ConnectionConfig | PlainMessage | undefined, b: ConnectionConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(ConnectionConfig, a, b); - } -} - -/** - * Configures a connection to a directory available on the local file system - * - * @generated from message mgmt.v1alpha1.LocalDirectoryConnectionConfig - */ -export class LocalDirectoryConnectionConfig extends Message { - /** - * The absolute path to a directory that is available on the local file system to the API and Worker nodes - * - * @generated from field: string path = 1; - */ - path = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.LocalDirectoryConnectionConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "path", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): LocalDirectoryConnectionConfig { - return new LocalDirectoryConnectionConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): LocalDirectoryConnectionConfig { - return new LocalDirectoryConnectionConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): LocalDirectoryConnectionConfig { - return new LocalDirectoryConnectionConfig().fromJsonString(jsonString, options); - } - - static equals(a: LocalDirectoryConnectionConfig | PlainMessage | undefined, b: LocalDirectoryConnectionConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(LocalDirectoryConnectionConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresConnectionConfig - */ -export class PostgresConnectionConfig extends Message { - /** - * May provide either a raw string url, or a structured version - * - * @generated from oneof mgmt.v1alpha1.PostgresConnectionConfig.connection_config - */ - connectionConfig: { - /** - * @generated from field: string url = 1; - */ - value: string; - case: "url"; - } | { - /** - * @generated from field: mgmt.v1alpha1.PostgresConnection connection = 2; - */ - value: PostgresConnection; - case: "connection"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - /** - * Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet - * - * @generated from field: mgmt.v1alpha1.SSHTunnel tunnel = 3; - */ - tunnel?: SSHTunnel; - - /** - * @generated from field: mgmt.v1alpha1.SqlConnectionOptions connection_options = 4; - */ - connectionOptions?: SqlConnectionOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresConnectionConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "connection_config" }, - { no: 2, name: "connection", kind: "message", T: PostgresConnection, oneof: "connection_config" }, - { no: 3, name: "tunnel", kind: "message", T: SSHTunnel }, - { no: 4, name: "connection_options", kind: "message", T: SqlConnectionOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresConnectionConfig { - return new PostgresConnectionConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresConnectionConfig { - return new PostgresConnectionConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresConnectionConfig { - return new PostgresConnectionConfig().fromJsonString(jsonString, options); - } - - static equals(a: PostgresConnectionConfig | PlainMessage | undefined, b: PostgresConnectionConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresConnectionConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SqlConnectionOptions - */ -export class SqlConnectionOptions extends Message { - /** - * @generated from field: optional int32 max_connection_limit = 1; - */ - maxConnectionLimit?: number; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SqlConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "max_connection_limit", kind: "scalar", T: 5 /* ScalarType.INT32 */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SqlConnectionOptions { - return new SqlConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SqlConnectionOptions { - return new SqlConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SqlConnectionOptions { - return new SqlConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: SqlConnectionOptions | PlainMessage | undefined, b: SqlConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(SqlConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SSHTunnel - */ -export class SSHTunnel extends Message { - /** - * The host of the SSH server - * - * @generated from field: string host = 1; - */ - host = ""; - - /** - * The port of the SSH server, typically 22 - * - * @generated from field: int32 port = 2; - */ - port = 0; - - /** - * The user that will be used to authenticate - * - * @generated from field: string user = 3; - */ - user = ""; - - /** - * Optionally provide the public key of the known host of the SSH tunnel that we are connecting to. - * If this is not provided, the server will blindly connect to the host with the given credentials. - * This is not recommended for production use! - * - * @generated from field: optional string known_host_public_key = 4; - */ - knownHostPublicKey?: string; - - /** - * Provide the authentication required to successfully connect to the SSH server for tunneling - * - * @generated from field: mgmt.v1alpha1.SSHAuthentication authentication = 5; - */ - authentication?: SSHAuthentication; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SSHTunnel"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, - { no: 3, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "known_host_public_key", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 5, name: "authentication", kind: "message", T: SSHAuthentication }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SSHTunnel { - return new SSHTunnel().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SSHTunnel { - return new SSHTunnel().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SSHTunnel { - return new SSHTunnel().fromJsonString(jsonString, options); - } - - static equals(a: SSHTunnel | PlainMessage | undefined, b: SSHTunnel | PlainMessage | undefined): boolean { - return proto3.util.equals(SSHTunnel, a, b); - } -} - -/** - * SSH Authentication - * - * @generated from message mgmt.v1alpha1.SSHAuthentication - */ -export class SSHAuthentication extends Message { - /** - * @generated from oneof mgmt.v1alpha1.SSHAuthentication.auth_config - */ - authConfig: { - /** - * @generated from field: mgmt.v1alpha1.SSHPassphrase passphrase = 1; - */ - value: SSHPassphrase; - case: "passphrase"; - } | { - /** - * @generated from field: mgmt.v1alpha1.SSHPrivateKey private_key = 2; - */ - value: SSHPrivateKey; - case: "privateKey"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SSHAuthentication"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "passphrase", kind: "message", T: SSHPassphrase, oneof: "auth_config" }, - { no: 2, name: "private_key", kind: "message", T: SSHPrivateKey, oneof: "auth_config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SSHAuthentication { - return new SSHAuthentication().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SSHAuthentication { - return new SSHAuthentication().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SSHAuthentication { - return new SSHAuthentication().fromJsonString(jsonString, options); - } - - static equals(a: SSHAuthentication | PlainMessage | undefined, b: SSHAuthentication | PlainMessage | undefined): boolean { - return proto3.util.equals(SSHAuthentication, a, b); - } -} - -/** - * Contains the configuration needed to retrieve the SSH passphrase for the tunnel - * - * @generated from message mgmt.v1alpha1.SSHPassphrase - */ -export class SSHPassphrase extends Message { - /** - * The password to be used to connect to the SSH server - * - * eventually we can expand this to allow pulling from other sources. - * - * @generated from field: string value = 1; - */ - value = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SSHPassphrase"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "value", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SSHPassphrase { - return new SSHPassphrase().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SSHPassphrase { - return new SSHPassphrase().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SSHPassphrase { - return new SSHPassphrase().fromJsonString(jsonString, options); - } - - static equals(a: SSHPassphrase | PlainMessage | undefined, b: SSHPassphrase | PlainMessage | undefined): boolean { - return proto3.util.equals(SSHPassphrase, a, b); - } -} - -/** - * Contains the configuration needed to retrieve the SSH private key for the tunnel - * - * @generated from message mgmt.v1alpha1.SSHPrivateKey - */ -export class SSHPrivateKey extends Message { - /** - * The private key in PEM format. - * - * @generated from field: string value = 1; - */ - value = ""; - - /** - * If the private key is encrypted, this value should decrypt it. - * - * @generated from field: optional string passphrase = 2; - */ - passphrase?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SSHPrivateKey"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "value", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "passphrase", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SSHPrivateKey { - return new SSHPrivateKey().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SSHPrivateKey { - return new SSHPrivateKey().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SSHPrivateKey { - return new SSHPrivateKey().fromJsonString(jsonString, options); - } - - static equals(a: SSHPrivateKey | PlainMessage | undefined, b: SSHPrivateKey | PlainMessage | undefined): boolean { - return proto3.util.equals(SSHPrivateKey, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresConnection - */ -export class PostgresConnection extends Message { - /** - * @generated from field: string host = 1; - */ - host = ""; - - /** - * @generated from field: int32 port = 2; - */ - port = 0; - - /** - * @generated from field: string name = 3; - */ - name = ""; - - /** - * @generated from field: string user = 4; - */ - user = ""; - - /** - * @generated from field: string pass = 5; - */ - pass = ""; - - /** - * @generated from field: optional string ssl_mode = 6; - */ - sslMode?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresConnection"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, - { no: 3, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "pass", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 6, name: "ssl_mode", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresConnection { - return new PostgresConnection().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresConnection { - return new PostgresConnection().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresConnection { - return new PostgresConnection().fromJsonString(jsonString, options); - } - - static equals(a: PostgresConnection | PlainMessage | undefined, b: PostgresConnection | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresConnection, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlConnection - */ -export class MysqlConnection extends Message { - /** - * @generated from field: string user = 1; - */ - user = ""; - - /** - * @generated from field: string pass = 2; - */ - pass = ""; - - /** - * @generated from field: string protocol = 3; - */ - protocol = ""; - - /** - * @generated from field: string host = 4; - */ - host = ""; - - /** - * @generated from field: int32 port = 5; - */ - port = 0; - - /** - * @generated from field: string name = 6; - */ - name = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlConnection"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "pass", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "protocol", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, - { no: 6, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlConnection { - return new MysqlConnection().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlConnection { - return new MysqlConnection().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlConnection { - return new MysqlConnection().fromJsonString(jsonString, options); - } - - static equals(a: MysqlConnection | PlainMessage | undefined, b: MysqlConnection | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlConnection, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlConnectionConfig - */ -export class MysqlConnectionConfig extends Message { - /** - * May provide either a raw string url, or a structured version - * - * @generated from oneof mgmt.v1alpha1.MysqlConnectionConfig.connection_config - */ - connectionConfig: { - /** - * @generated from field: string url = 1; - */ - value: string; - case: "url"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlConnection connection = 2; - */ - value: MysqlConnection; - case: "connection"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - /** - * Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet - * - * @generated from field: mgmt.v1alpha1.SSHTunnel tunnel = 3; - */ - tunnel?: SSHTunnel; - - /** - * @generated from field: mgmt.v1alpha1.SqlConnectionOptions connection_options = 4; - */ - connectionOptions?: SqlConnectionOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlConnectionConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "connection_config" }, - { no: 2, name: "connection", kind: "message", T: MysqlConnection, oneof: "connection_config" }, - { no: 3, name: "tunnel", kind: "message", T: SSHTunnel }, - { no: 4, name: "connection_options", kind: "message", T: SqlConnectionOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlConnectionConfig { - return new MysqlConnectionConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlConnectionConfig { - return new MysqlConnectionConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlConnectionConfig { - return new MysqlConnectionConfig().fromJsonString(jsonString, options); - } - - static equals(a: MysqlConnectionConfig | PlainMessage | undefined, b: MysqlConnectionConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlConnectionConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AwsS3ConnectionConfig - */ -export class AwsS3ConnectionConfig extends Message { - /** - * @generated from field: string bucket_arn = 1 [deprecated = true]; - * @deprecated - */ - bucketArn = ""; - - /** - * @generated from field: optional string path_prefix = 2; - */ - pathPrefix?: string; - - /** - * @generated from field: optional mgmt.v1alpha1.AwsS3Credentials credentials = 3; - */ - credentials?: AwsS3Credentials; - - /** - * @generated from field: optional string region = 4; - */ - region?: string; - - /** - * @generated from field: optional string endpoint = 5; - */ - endpoint?: string; - - /** - * @generated from field: string bucket = 6; - */ - bucket = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3ConnectionConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "bucket_arn", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "path_prefix", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 3, name: "credentials", kind: "message", T: AwsS3Credentials, opt: true }, - { no: 4, name: "region", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 5, name: "endpoint", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 6, name: "bucket", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3ConnectionConfig { - return new AwsS3ConnectionConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3ConnectionConfig { - return new AwsS3ConnectionConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3ConnectionConfig { - return new AwsS3ConnectionConfig().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3ConnectionConfig | PlainMessage | undefined, b: AwsS3ConnectionConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3ConnectionConfig, a, b); - } -} - -/** - * S3 Credentials that are used by the worker process. - * Note: this may be optionally provided if the worker that is being hosted has environment credentials to the S3 bucket instead. - * - * @generated from message mgmt.v1alpha1.AwsS3Credentials - */ -export class AwsS3Credentials extends Message { - /** - * @generated from field: optional string profile = 1; - */ - profile?: string; - - /** - * @generated from field: optional string access_key_id = 2; - */ - accessKeyId?: string; - - /** - * @generated from field: optional string secret_access_key = 3; - */ - secretAccessKey?: string; - - /** - * @generated from field: optional string session_token = 4; - */ - sessionToken?: string; - - /** - * @generated from field: optional bool from_ec2_role = 5; - */ - fromEc2Role?: boolean; - - /** - * @generated from field: optional string role_arn = 6; - */ - roleArn?: string; - - /** - * @generated from field: optional string role_external_id = 7; - */ - roleExternalId?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3Credentials"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "profile", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 2, name: "access_key_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 3, name: "secret_access_key", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 4, name: "session_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 5, name: "from_ec2_role", kind: "scalar", T: 8 /* ScalarType.BOOL */, opt: true }, - { no: 6, name: "role_arn", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 7, name: "role_external_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3Credentials { - return new AwsS3Credentials().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3Credentials { - return new AwsS3Credentials().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3Credentials { - return new AwsS3Credentials().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3Credentials | PlainMessage | undefined, b: AwsS3Credentials | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3Credentials, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsConnectionNameAvailableRequest - */ -export class IsConnectionNameAvailableRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string connection_name = 2; - */ - connectionName = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsConnectionNameAvailableRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "connection_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsConnectionNameAvailableRequest { - return new IsConnectionNameAvailableRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsConnectionNameAvailableRequest { - return new IsConnectionNameAvailableRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsConnectionNameAvailableRequest { - return new IsConnectionNameAvailableRequest().fromJsonString(jsonString, options); - } - - static equals(a: IsConnectionNameAvailableRequest | PlainMessage | undefined, b: IsConnectionNameAvailableRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(IsConnectionNameAvailableRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsConnectionNameAvailableResponse - */ -export class IsConnectionNameAvailableResponse extends Message { - /** - * @generated from field: bool is_available = 1; - */ - isAvailable = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsConnectionNameAvailableResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsConnectionNameAvailableResponse { - return new IsConnectionNameAvailableResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsConnectionNameAvailableResponse { - return new IsConnectionNameAvailableResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsConnectionNameAvailableResponse { - return new IsConnectionNameAvailableResponse().fromJsonString(jsonString, options); - } - - static equals(a: IsConnectionNameAvailableResponse | PlainMessage | undefined, b: IsConnectionNameAvailableResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(IsConnectionNameAvailableResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckSqlQueryRequest - */ -export class CheckSqlQueryRequest extends Message { - /** - * The connection id that the query will be checked against - * - * @generated from field: string id = 1; - */ - id = ""; - - /** - * The full query that will be run through a PREPARE statement - * - * @generated from field: string query = 2; - */ - query = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckSqlQueryRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "query", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckSqlQueryRequest { - return new CheckSqlQueryRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckSqlQueryRequest { - return new CheckSqlQueryRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckSqlQueryRequest { - return new CheckSqlQueryRequest().fromJsonString(jsonString, options); - } - - static equals(a: CheckSqlQueryRequest | PlainMessage | undefined, b: CheckSqlQueryRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckSqlQueryRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CheckSqlQueryResponse - */ -export class CheckSqlQueryResponse extends Message { - /** - * The query is run through PREPARE. Returns valid if it correctly compiled - * - * @generated from field: bool is_valid = 1; - */ - isValid = false; - - /** - * The error message returned by the sql client if the prepare did not return successfully - * - * @generated from field: optional string erorr_message = 2; - */ - erorrMessage?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CheckSqlQueryResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "erorr_message", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CheckSqlQueryResponse { - return new CheckSqlQueryResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CheckSqlQueryResponse { - return new CheckSqlQueryResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CheckSqlQueryResponse { - return new CheckSqlQueryResponse().fromJsonString(jsonString, options); - } - - static equals(a: CheckSqlQueryResponse | PlainMessage | undefined, b: CheckSqlQueryResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CheckSqlQueryResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts deleted file mode 100644 index 326c87142d..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts +++ /dev/null @@ -1,265 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/job.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { CancelJobRunRequest, CancelJobRunResponse, CreateJobDestinationConnectionsRequest, CreateJobDestinationConnectionsResponse, CreateJobRequest, CreateJobResponse, CreateJobRunRequest, CreateJobRunResponse, DeleteJobDestinationConnectionRequest, DeleteJobDestinationConnectionResponse, DeleteJobRequest, DeleteJobResponse, DeleteJobRunRequest, DeleteJobRunResponse, GetJobNextRunsRequest, GetJobNextRunsResponse, GetJobRecentRunsRequest, GetJobRecentRunsResponse, GetJobRequest, GetJobResponse, GetJobRunEventsRequest, GetJobRunEventsResponse, GetJobRunLogsStreamRequest, GetJobRunLogsStreamResponse, GetJobRunRequest, GetJobRunResponse, GetJobRunsRequest, GetJobRunsResponse, GetJobsRequest, GetJobsResponse, GetJobStatusesRequest, GetJobStatusesResponse, GetJobStatusRequest, GetJobStatusResponse, IsJobNameAvailableRequest, IsJobNameAvailableResponse, PauseJobRequest, PauseJobResponse, SetJobSourceSqlConnectionSubsetsRequest, SetJobSourceSqlConnectionSubsetsResponse, SetJobSyncOptionsRequest, SetJobSyncOptionsResponse, SetJobWorkflowOptionsRequest, SetJobWorkflowOptionsResponse, TerminateJobRunRequest, TerminateJobRunResponse, UpdateJobDestinationConnectionRequest, UpdateJobDestinationConnectionResponse, UpdateJobScheduleRequest, UpdateJobScheduleResponse, UpdateJobSourceConnectionRequest, UpdateJobSourceConnectionResponse } from "./job_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * @generated from service mgmt.v1alpha1.JobService - */ -export const JobService = { - typeName: "mgmt.v1alpha1.JobService", - methods: { - /** - * @generated from rpc mgmt.v1alpha1.JobService.GetJobs - */ - getJobs: { - name: "GetJobs", - I: GetJobsRequest, - O: GetJobsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.GetJob - */ - getJob: { - name: "GetJob", - I: GetJobRequest, - O: GetJobResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.CreateJob - */ - createJob: { - name: "CreateJob", - I: CreateJobRequest, - O: CreateJobResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.DeleteJob - */ - deleteJob: { - name: "DeleteJob", - I: DeleteJobRequest, - O: DeleteJobResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.IsJobNameAvailable - */ - isJobNameAvailable: { - name: "IsJobNameAvailable", - I: IsJobNameAvailableRequest, - O: IsJobNameAvailableResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobSchedule - */ - updateJobSchedule: { - name: "UpdateJobSchedule", - I: UpdateJobScheduleRequest, - O: UpdateJobScheduleResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobSourceConnection - */ - updateJobSourceConnection: { - name: "UpdateJobSourceConnection", - I: UpdateJobSourceConnectionRequest, - O: UpdateJobSourceConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.SetJobSourceSqlConnectionSubsets - */ - setJobSourceSqlConnectionSubsets: { - name: "SetJobSourceSqlConnectionSubsets", - I: SetJobSourceSqlConnectionSubsetsRequest, - O: SetJobSourceSqlConnectionSubsetsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobDestinationConnection - */ - updateJobDestinationConnection: { - name: "UpdateJobDestinationConnection", - I: UpdateJobDestinationConnectionRequest, - O: UpdateJobDestinationConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.DeleteJobDestinationConnection - */ - deleteJobDestinationConnection: { - name: "DeleteJobDestinationConnection", - I: DeleteJobDestinationConnectionRequest, - O: DeleteJobDestinationConnectionResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.CreateJobDestinationConnections - */ - createJobDestinationConnections: { - name: "CreateJobDestinationConnections", - I: CreateJobDestinationConnectionsRequest, - O: CreateJobDestinationConnectionsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.PauseJob - */ - pauseJob: { - name: "PauseJob", - I: PauseJobRequest, - O: PauseJobResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a list of recently invoked job runs baseds on the Temporal cron scheduler. This will return a list of job runs that include archived runs - * - * @generated from rpc mgmt.v1alpha1.JobService.GetJobRecentRuns - */ - getJobRecentRuns: { - name: "GetJobRecentRuns", - I: GetJobRecentRunsRequest, - O: GetJobRecentRunsResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a list of runs that are scheduled for execution based on the Temporal cron scheduler. - * - * @generated from rpc mgmt.v1alpha1.JobService.GetJobNextRuns - */ - getJobNextRuns: { - name: "GetJobNextRuns", - I: GetJobNextRunsRequest, - O: GetJobNextRunsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.GetJobStatus - */ - getJobStatus: { - name: "GetJobStatus", - I: GetJobStatusRequest, - O: GetJobStatusResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.GetJobStatuses - */ - getJobStatuses: { - name: "GetJobStatuses", - I: GetJobStatusesRequest, - O: GetJobStatusesResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a list of job runs by either account or job - * - * @generated from rpc mgmt.v1alpha1.JobService.GetJobRuns - */ - getJobRuns: { - name: "GetJobRuns", - I: GetJobRunsRequest, - O: GetJobRunsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.GetJobRunEvents - */ - getJobRunEvents: { - name: "GetJobRunEvents", - I: GetJobRunEventsRequest, - O: GetJobRunEventsResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a specific job run, along with any of its pending activities - * - * @generated from rpc mgmt.v1alpha1.JobService.GetJobRun - */ - getJobRun: { - name: "GetJobRun", - I: GetJobRunRequest, - O: GetJobRunResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.DeleteJobRun - */ - deleteJobRun: { - name: "DeleteJobRun", - I: DeleteJobRunRequest, - O: DeleteJobRunResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.CreateJobRun - */ - createJobRun: { - name: "CreateJobRun", - I: CreateJobRunRequest, - O: CreateJobRunResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.CancelJobRun - */ - cancelJobRun: { - name: "CancelJobRun", - I: CancelJobRunRequest, - O: CancelJobRunResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.JobService.TerminateJobRun - */ - terminateJobRun: { - name: "TerminateJobRun", - I: TerminateJobRunRequest, - O: TerminateJobRunResponse, - kind: MethodKind.Unary, - }, - /** - * Returns a stream of logs from the worker nodes that pertain to a specific job run - * - * @generated from rpc mgmt.v1alpha1.JobService.GetJobRunLogsStream - */ - getJobRunLogsStream: { - name: "GetJobRunLogsStream", - I: GetJobRunLogsStreamRequest, - O: GetJobRunLogsStreamResponse, - kind: MethodKind.ServerStreaming, - }, - /** - * Set any job workflow options. Must provide entire object as is it will fully override the previous configuration - * - * @generated from rpc mgmt.v1alpha1.JobService.SetJobWorkflowOptions - */ - setJobWorkflowOptions: { - name: "SetJobWorkflowOptions", - I: SetJobWorkflowOptionsRequest, - O: SetJobWorkflowOptionsResponse, - kind: MethodKind.Unary, - }, - /** - * Set the job sync options. Must provide entire object as it will fully override the previous configuration - * - * @generated from rpc mgmt.v1alpha1.JobService.SetJobSyncOptions - */ - setJobSyncOptions: { - name: "SetJobSyncOptions", - I: SetJobSyncOptionsRequest, - O: SetJobSyncOptionsResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts deleted file mode 100644 index caaa9d2141..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts +++ /dev/null @@ -1,4384 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/job.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; -import { TransformerConfig, TransformerSource } from "./transformer_pb.js"; - -/** - * @generated from enum mgmt.v1alpha1.JobStatus - */ -export enum JobStatus { - /** - * @generated from enum value: JOB_STATUS_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: JOB_STATUS_ENABLED = 1; - */ - ENABLED = 1, - - /** - * @generated from enum value: JOB_STATUS_PAUSED = 3; - */ - PAUSED = 3, - - /** - * @generated from enum value: JOB_STATUS_DISABLED = 4; - */ - DISABLED = 4, -} -// Retrieve enum metadata with: proto3.getEnumType(JobStatus) -proto3.util.setEnumType(JobStatus, "mgmt.v1alpha1.JobStatus", [ - { no: 0, name: "JOB_STATUS_UNSPECIFIED" }, - { no: 1, name: "JOB_STATUS_ENABLED" }, - { no: 3, name: "JOB_STATUS_PAUSED" }, - { no: 4, name: "JOB_STATUS_DISABLED" }, -]); - -/** - * @generated from enum mgmt.v1alpha1.ActivityStatus - */ -export enum ActivityStatus { - /** - * @generated from enum value: ACTIVITY_STATUS_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: ACTIVITY_STATUS_SCHEDULED = 1; - */ - SCHEDULED = 1, - - /** - * @generated from enum value: ACTIVITY_STATUS_STARTED = 2; - */ - STARTED = 2, - - /** - * @generated from enum value: ACTIVITY_STATUS_CANCELED = 3; - */ - CANCELED = 3, - - /** - * @generated from enum value: ACTIVITY_STATUS_FAILED = 4; - */ - FAILED = 4, -} -// Retrieve enum metadata with: proto3.getEnumType(ActivityStatus) -proto3.util.setEnumType(ActivityStatus, "mgmt.v1alpha1.ActivityStatus", [ - { no: 0, name: "ACTIVITY_STATUS_UNSPECIFIED" }, - { no: 1, name: "ACTIVITY_STATUS_SCHEDULED" }, - { no: 2, name: "ACTIVITY_STATUS_STARTED" }, - { no: 3, name: "ACTIVITY_STATUS_CANCELED" }, - { no: 4, name: "ACTIVITY_STATUS_FAILED" }, -]); - -/** - * An enumeration of job run statuses. - * - * @generated from enum mgmt.v1alpha1.JobRunStatus - */ -export enum JobRunStatus { - /** - * if the job run status is unknown - * - * @generated from enum value: JOB_RUN_STATUS_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * the run is pending and has not started yet - * - * @generated from enum value: JOB_RUN_STATUS_PENDING = 1; - */ - PENDING = 1, - - /** - * the run is currently in progress - * - * @generated from enum value: JOB_RUN_STATUS_RUNNING = 2; - */ - RUNNING = 2, - - /** - * the run has successfully completed - * - * @generated from enum value: JOB_RUN_STATUS_COMPLETE = 3; - */ - COMPLETE = 3, - - /** - * the run ended with an error - * - * @generated from enum value: JOB_RUN_STATUS_ERROR = 4; - */ - ERROR = 4, - - /** - * the run was cancelled - * - * @generated from enum value: JOB_RUN_STATUS_CANCELED = 5; - */ - CANCELED = 5, - - /** - * the run was terminated - * - * @generated from enum value: JOB_RUN_STATUS_TERMINATED = 6; - */ - TERMINATED = 6, - - /** - * the run ended in failure - * - * @generated from enum value: JOB_RUN_STATUS_FAILED = 7; - */ - FAILED = 7, - - /** - * the run was ended pre-maturely due to timeout - * - * @generated from enum value: JOB_RUN_STATUS_TIMED_OUT = 8; - */ - TIMED_OUT = 8, -} -// Retrieve enum metadata with: proto3.getEnumType(JobRunStatus) -proto3.util.setEnumType(JobRunStatus, "mgmt.v1alpha1.JobRunStatus", [ - { no: 0, name: "JOB_RUN_STATUS_UNSPECIFIED" }, - { no: 1, name: "JOB_RUN_STATUS_PENDING" }, - { no: 2, name: "JOB_RUN_STATUS_RUNNING" }, - { no: 3, name: "JOB_RUN_STATUS_COMPLETE" }, - { no: 4, name: "JOB_RUN_STATUS_ERROR" }, - { no: 5, name: "JOB_RUN_STATUS_CANCELED" }, - { no: 6, name: "JOB_RUN_STATUS_TERMINATED" }, - { no: 7, name: "JOB_RUN_STATUS_FAILED" }, - { no: 8, name: "JOB_RUN_STATUS_TIMED_OUT" }, -]); - -/** - * @generated from enum mgmt.v1alpha1.LogWindow - */ -export enum LogWindow { - /** - * @generated from enum value: LOG_WINDOW_NO_TIME_UNSPECIFIED = 0; - */ - NO_TIME_UNSPECIFIED = 0, - - /** - * @generated from enum value: LOG_WINDOW_FIFTEEN_MIN = 1; - */ - FIFTEEN_MIN = 1, - - /** - * @generated from enum value: LOG_WINDOW_ONE_HOUR = 2; - */ - ONE_HOUR = 2, - - /** - * @generated from enum value: LOG_WINDOW_ONE_DAY = 3; - */ - ONE_DAY = 3, -} -// Retrieve enum metadata with: proto3.getEnumType(LogWindow) -proto3.util.setEnumType(LogWindow, "mgmt.v1alpha1.LogWindow", [ - { no: 0, name: "LOG_WINDOW_NO_TIME_UNSPECIFIED" }, - { no: 1, name: "LOG_WINDOW_FIFTEEN_MIN" }, - { no: 2, name: "LOG_WINDOW_ONE_HOUR" }, - { no: 3, name: "LOG_WINDOW_ONE_DAY" }, -]); - -/** - * @generated from enum mgmt.v1alpha1.LogLevel - */ -export enum LogLevel { - /** - * @generated from enum value: LOG_LEVEL_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: LOG_LEVEL_DEBUG = 1; - */ - DEBUG = 1, - - /** - * @generated from enum value: LOG_LEVEL_INFO = 2; - */ - INFO = 2, - - /** - * @generated from enum value: LOG_LEVEL_WARN = 3; - */ - WARN = 3, - - /** - * @generated from enum value: LOG_LEVEL_ERROR = 4; - */ - ERROR = 4, -} -// Retrieve enum metadata with: proto3.getEnumType(LogLevel) -proto3.util.setEnumType(LogLevel, "mgmt.v1alpha1.LogLevel", [ - { no: 0, name: "LOG_LEVEL_UNSPECIFIED" }, - { no: 1, name: "LOG_LEVEL_DEBUG" }, - { no: 2, name: "LOG_LEVEL_INFO" }, - { no: 3, name: "LOG_LEVEL_WARN" }, - { no: 4, name: "LOG_LEVEL_ERROR" }, -]); - -/** - * @generated from message mgmt.v1alpha1.GetJobsRequest - */ -export class GetJobsRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobsRequest { - return new GetJobsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobsRequest { - return new GetJobsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobsRequest { - return new GetJobsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobsRequest | PlainMessage | undefined, b: GetJobsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobsResponse - */ -export class GetJobsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.Job jobs = 1; - */ - jobs: Job[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "jobs", kind: "message", T: Job, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobsResponse { - return new GetJobsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobsResponse { - return new GetJobsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobsResponse { - return new GetJobsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobsResponse | PlainMessage | undefined, b: GetJobsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobSource - */ -export class JobSource extends Message { - /** - * @generated from field: mgmt.v1alpha1.JobSourceOptions options = 1; - */ - options?: JobSourceOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobSource"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "options", kind: "message", T: JobSourceOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobSource { - return new JobSource().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobSource { - return new JobSource().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobSource { - return new JobSource().fromJsonString(jsonString, options); - } - - static equals(a: JobSource | PlainMessage | undefined, b: JobSource | PlainMessage | undefined): boolean { - return proto3.util.equals(JobSource, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobSourceOptions - */ -export class JobSourceOptions extends Message { - /** - * @generated from oneof mgmt.v1alpha1.JobSourceOptions.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.PostgresSourceConnectionOptions postgres = 1; - */ - value: PostgresSourceConnectionOptions; - case: "postgres"; - } | { - /** - * @generated from field: mgmt.v1alpha1.AwsS3SourceConnectionOptions aws_s3 = 2; - */ - value: AwsS3SourceConnectionOptions; - case: "awsS3"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlSourceConnectionOptions mysql = 3; - */ - value: MysqlSourceConnectionOptions; - case: "mysql"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateSourceOptions generate = 4; - */ - value: GenerateSourceOptions; - case: "generate"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobSourceOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "postgres", kind: "message", T: PostgresSourceConnectionOptions, oneof: "config" }, - { no: 2, name: "aws_s3", kind: "message", T: AwsS3SourceConnectionOptions, oneof: "config" }, - { no: 3, name: "mysql", kind: "message", T: MysqlSourceConnectionOptions, oneof: "config" }, - { no: 4, name: "generate", kind: "message", T: GenerateSourceOptions, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobSourceOptions { - return new JobSourceOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobSourceOptions { - return new JobSourceOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobSourceOptions { - return new JobSourceOptions().fromJsonString(jsonString, options); - } - - static equals(a: JobSourceOptions | PlainMessage | undefined, b: JobSourceOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(JobSourceOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobDestination - */ -export class CreateJobDestination extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 2; - */ - options?: JobDestinationOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobDestination"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "options", kind: "message", T: JobDestinationOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestination { - return new CreateJobDestination().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestination { - return new CreateJobDestination().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobDestination { - return new CreateJobDestination().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobDestination | PlainMessage | undefined, b: CreateJobDestination | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobDestination, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobDestination - */ -export class JobDestination extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 2; - */ - options?: JobDestinationOptions; - - /** - * @generated from field: string id = 3; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobDestination"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "options", kind: "message", T: JobDestinationOptions }, - { no: 3, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobDestination { - return new JobDestination().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobDestination { - return new JobDestination().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobDestination { - return new JobDestination().fromJsonString(jsonString, options); - } - - static equals(a: JobDestination | PlainMessage | undefined, b: JobDestination | PlainMessage | undefined): boolean { - return proto3.util.equals(JobDestination, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateSourceOptions - */ -export class GenerateSourceOptions extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.GenerateSourceSchemaOption schemas = 1; - */ - schemas: GenerateSourceSchemaOption[] = []; - - /** - * @generated from field: optional string fk_source_connection_id = 3; - */ - fkSourceConnectionId?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateSourceOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schemas", kind: "message", T: GenerateSourceSchemaOption, repeated: true }, - { no: 3, name: "fk_source_connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceOptions { - return new GenerateSourceOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceOptions { - return new GenerateSourceOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateSourceOptions { - return new GenerateSourceOptions().fromJsonString(jsonString, options); - } - - static equals(a: GenerateSourceOptions | PlainMessage | undefined, b: GenerateSourceOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateSourceOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateSourceSchemaOption - */ -export class GenerateSourceSchemaOption extends Message { - /** - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * @generated from field: repeated mgmt.v1alpha1.GenerateSourceTableOption tables = 2; - */ - tables: GenerateSourceTableOption[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateSourceSchemaOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "tables", kind: "message", T: GenerateSourceTableOption, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceSchemaOption { - return new GenerateSourceSchemaOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceSchemaOption { - return new GenerateSourceSchemaOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateSourceSchemaOption { - return new GenerateSourceSchemaOption().fromJsonString(jsonString, options); - } - - static equals(a: GenerateSourceSchemaOption | PlainMessage | undefined, b: GenerateSourceSchemaOption | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateSourceSchemaOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateSourceTableOption - */ -export class GenerateSourceTableOption extends Message { - /** - * @generated from field: string table = 1; - */ - table = ""; - - /** - * @generated from field: int64 row_count = 2; - */ - rowCount = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateSourceTableOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "row_count", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceTableOption { - return new GenerateSourceTableOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceTableOption { - return new GenerateSourceTableOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateSourceTableOption { - return new GenerateSourceTableOption().fromJsonString(jsonString, options); - } - - static equals(a: GenerateSourceTableOption | PlainMessage | undefined, b: GenerateSourceTableOption | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateSourceTableOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresSourceConnectionOptions - */ -export class PostgresSourceConnectionOptions extends Message { - /** - * @generated from field: bool halt_on_new_column_addition = 1; - */ - haltOnNewColumnAddition = false; - - /** - * @generated from field: repeated mgmt.v1alpha1.PostgresSourceSchemaOption schemas = 2; - */ - schemas: PostgresSourceSchemaOption[] = []; - - /** - * @generated from field: string connection_id = 3; - */ - connectionId = ""; - - /** - * @generated from field: bool subset_by_foreign_key_constraints = 4; - */ - subsetByForeignKeyConstraints = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresSourceConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "halt_on_new_column_addition", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "schemas", kind: "message", T: PostgresSourceSchemaOption, repeated: true }, - { no: 3, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceConnectionOptions { - return new PostgresSourceConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceConnectionOptions { - return new PostgresSourceConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresSourceConnectionOptions { - return new PostgresSourceConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: PostgresSourceConnectionOptions | PlainMessage | undefined, b: PostgresSourceConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresSourceConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresSourceSchemaOption - */ -export class PostgresSourceSchemaOption extends Message { - /** - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * @generated from field: repeated mgmt.v1alpha1.PostgresSourceTableOption tables = 2; - */ - tables: PostgresSourceTableOption[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresSourceSchemaOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "tables", kind: "message", T: PostgresSourceTableOption, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceSchemaOption { - return new PostgresSourceSchemaOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceSchemaOption { - return new PostgresSourceSchemaOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresSourceSchemaOption { - return new PostgresSourceSchemaOption().fromJsonString(jsonString, options); - } - - static equals(a: PostgresSourceSchemaOption | PlainMessage | undefined, b: PostgresSourceSchemaOption | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresSourceSchemaOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresSourceTableOption - */ -export class PostgresSourceTableOption extends Message { - /** - * @generated from field: string table = 1; - */ - table = ""; - - /** - * @generated from field: optional string where_clause = 2; - */ - whereClause?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresSourceTableOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "where_clause", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceTableOption { - return new PostgresSourceTableOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceTableOption { - return new PostgresSourceTableOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresSourceTableOption { - return new PostgresSourceTableOption().fromJsonString(jsonString, options); - } - - static equals(a: PostgresSourceTableOption | PlainMessage | undefined, b: PostgresSourceTableOption | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresSourceTableOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlSourceConnectionOptions - */ -export class MysqlSourceConnectionOptions extends Message { - /** - * @generated from field: bool halt_on_new_column_addition = 1; - */ - haltOnNewColumnAddition = false; - - /** - * @generated from field: repeated mgmt.v1alpha1.MysqlSourceSchemaOption schemas = 2; - */ - schemas: MysqlSourceSchemaOption[] = []; - - /** - * @generated from field: string connection_id = 3; - */ - connectionId = ""; - - /** - * @generated from field: bool subset_by_foreign_key_constraints = 4; - */ - subsetByForeignKeyConstraints = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlSourceConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "halt_on_new_column_addition", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "schemas", kind: "message", T: MysqlSourceSchemaOption, repeated: true }, - { no: 3, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceConnectionOptions { - return new MysqlSourceConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceConnectionOptions { - return new MysqlSourceConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlSourceConnectionOptions { - return new MysqlSourceConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: MysqlSourceConnectionOptions | PlainMessage | undefined, b: MysqlSourceConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlSourceConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlSourceSchemaOption - */ -export class MysqlSourceSchemaOption extends Message { - /** - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * @generated from field: repeated mgmt.v1alpha1.MysqlSourceTableOption tables = 2; - */ - tables: MysqlSourceTableOption[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlSourceSchemaOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "tables", kind: "message", T: MysqlSourceTableOption, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceSchemaOption { - return new MysqlSourceSchemaOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceSchemaOption { - return new MysqlSourceSchemaOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlSourceSchemaOption { - return new MysqlSourceSchemaOption().fromJsonString(jsonString, options); - } - - static equals(a: MysqlSourceSchemaOption | PlainMessage | undefined, b: MysqlSourceSchemaOption | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlSourceSchemaOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlSourceTableOption - */ -export class MysqlSourceTableOption extends Message { - /** - * @generated from field: string table = 1; - */ - table = ""; - - /** - * @generated from field: optional string where_clause = 2; - */ - whereClause?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlSourceTableOption"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "where_clause", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceTableOption { - return new MysqlSourceTableOption().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceTableOption { - return new MysqlSourceTableOption().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlSourceTableOption { - return new MysqlSourceTableOption().fromJsonString(jsonString, options); - } - - static equals(a: MysqlSourceTableOption | PlainMessage | undefined, b: MysqlSourceTableOption | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlSourceTableOption, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AwsS3SourceConnectionOptions - */ -export class AwsS3SourceConnectionOptions extends Message { - /** - * @generated from field: string connection_id = 1; - */ - connectionId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3SourceConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3SourceConnectionOptions { - return new AwsS3SourceConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3SourceConnectionOptions { - return new AwsS3SourceConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3SourceConnectionOptions { - return new AwsS3SourceConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3SourceConnectionOptions | PlainMessage | undefined, b: AwsS3SourceConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3SourceConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobDestinationOptions - */ -export class JobDestinationOptions extends Message { - /** - * @generated from oneof mgmt.v1alpha1.JobDestinationOptions.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.PostgresDestinationConnectionOptions postgres_options = 1; - */ - value: PostgresDestinationConnectionOptions; - case: "postgresOptions"; - } | { - /** - * @generated from field: mgmt.v1alpha1.AwsS3DestinationConnectionOptions aws_s3_options = 2; - */ - value: AwsS3DestinationConnectionOptions; - case: "awsS3Options"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlDestinationConnectionOptions mysql_options = 3; - */ - value: MysqlDestinationConnectionOptions; - case: "mysqlOptions"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobDestinationOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "postgres_options", kind: "message", T: PostgresDestinationConnectionOptions, oneof: "config" }, - { no: 2, name: "aws_s3_options", kind: "message", T: AwsS3DestinationConnectionOptions, oneof: "config" }, - { no: 3, name: "mysql_options", kind: "message", T: MysqlDestinationConnectionOptions, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobDestinationOptions { - return new JobDestinationOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobDestinationOptions { - return new JobDestinationOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobDestinationOptions { - return new JobDestinationOptions().fromJsonString(jsonString, options); - } - - static equals(a: JobDestinationOptions | PlainMessage | undefined, b: JobDestinationOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(JobDestinationOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresDestinationConnectionOptions - */ -export class PostgresDestinationConnectionOptions extends Message { - /** - * @generated from field: mgmt.v1alpha1.PostgresTruncateTableConfig truncate_table = 1; - */ - truncateTable?: PostgresTruncateTableConfig; - - /** - * @generated from field: bool init_table_schema = 2; - */ - initTableSchema = false; - - /** - * @generated from field: mgmt.v1alpha1.PostgresOnConflictConfig on_conflict = 3; - */ - onConflict?: PostgresOnConflictConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresDestinationConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "truncate_table", kind: "message", T: PostgresTruncateTableConfig }, - { no: 2, name: "init_table_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "on_conflict", kind: "message", T: PostgresOnConflictConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresDestinationConnectionOptions { - return new PostgresDestinationConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresDestinationConnectionOptions { - return new PostgresDestinationConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresDestinationConnectionOptions { - return new PostgresDestinationConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: PostgresDestinationConnectionOptions | PlainMessage | undefined, b: PostgresDestinationConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresDestinationConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresOnConflictConfig - */ -export class PostgresOnConflictConfig extends Message { - /** - * @generated from field: bool do_nothing = 1; - */ - doNothing = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresOnConflictConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "do_nothing", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresOnConflictConfig { - return new PostgresOnConflictConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresOnConflictConfig { - return new PostgresOnConflictConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresOnConflictConfig { - return new PostgresOnConflictConfig().fromJsonString(jsonString, options); - } - - static equals(a: PostgresOnConflictConfig | PlainMessage | undefined, b: PostgresOnConflictConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresOnConflictConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresTruncateTableConfig - */ -export class PostgresTruncateTableConfig extends Message { - /** - * @generated from field: bool truncate_before_insert = 1; - */ - truncateBeforeInsert = false; - - /** - * @generated from field: bool cascade = 2; - */ - cascade = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresTruncateTableConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "cascade", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresTruncateTableConfig { - return new PostgresTruncateTableConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresTruncateTableConfig { - return new PostgresTruncateTableConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresTruncateTableConfig { - return new PostgresTruncateTableConfig().fromJsonString(jsonString, options); - } - - static equals(a: PostgresTruncateTableConfig | PlainMessage | undefined, b: PostgresTruncateTableConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresTruncateTableConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlDestinationConnectionOptions - */ -export class MysqlDestinationConnectionOptions extends Message { - /** - * @generated from field: mgmt.v1alpha1.MysqlTruncateTableConfig truncate_table = 1; - */ - truncateTable?: MysqlTruncateTableConfig; - - /** - * @generated from field: bool init_table_schema = 2; - */ - initTableSchema = false; - - /** - * @generated from field: mgmt.v1alpha1.MysqlOnConflictConfig on_conflict = 3; - */ - onConflict?: MysqlOnConflictConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlDestinationConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "truncate_table", kind: "message", T: MysqlTruncateTableConfig }, - { no: 2, name: "init_table_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "on_conflict", kind: "message", T: MysqlOnConflictConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlDestinationConnectionOptions { - return new MysqlDestinationConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlDestinationConnectionOptions { - return new MysqlDestinationConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlDestinationConnectionOptions { - return new MysqlDestinationConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: MysqlDestinationConnectionOptions | PlainMessage | undefined, b: MysqlDestinationConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlDestinationConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlTruncateTableConfig - */ -export class MysqlTruncateTableConfig extends Message { - /** - * @generated from field: bool truncate_before_insert = 1; - */ - truncateBeforeInsert = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlTruncateTableConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlTruncateTableConfig { - return new MysqlTruncateTableConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlTruncateTableConfig { - return new MysqlTruncateTableConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlTruncateTableConfig { - return new MysqlTruncateTableConfig().fromJsonString(jsonString, options); - } - - static equals(a: MysqlTruncateTableConfig | PlainMessage | undefined, b: MysqlTruncateTableConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlTruncateTableConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlOnConflictConfig - */ -export class MysqlOnConflictConfig extends Message { - /** - * @generated from field: bool do_nothing = 1; - */ - doNothing = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlOnConflictConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "do_nothing", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlOnConflictConfig { - return new MysqlOnConflictConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlOnConflictConfig { - return new MysqlOnConflictConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlOnConflictConfig { - return new MysqlOnConflictConfig().fromJsonString(jsonString, options); - } - - static equals(a: MysqlOnConflictConfig | PlainMessage | undefined, b: MysqlOnConflictConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlOnConflictConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AwsS3DestinationConnectionOptions - */ -export class AwsS3DestinationConnectionOptions extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AwsS3DestinationConnectionOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3DestinationConnectionOptions { - return new AwsS3DestinationConnectionOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3DestinationConnectionOptions { - return new AwsS3DestinationConnectionOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AwsS3DestinationConnectionOptions { - return new AwsS3DestinationConnectionOptions().fromJsonString(jsonString, options); - } - - static equals(a: AwsS3DestinationConnectionOptions | PlainMessage | undefined, b: AwsS3DestinationConnectionOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(AwsS3DestinationConnectionOptions, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobRequest - */ -export class CreateJobRequest extends Message { - /** - * The unique account identifier that this job will be associated with - * - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * The unique, friendly name of the job. This is unique per account - * - * @generated from field: string job_name = 2; - */ - jobName = ""; - - /** - * Optionally provide a cron schedule. Goes into effect if the job status is set to enabled - * - * @generated from field: optional string cron_schedule = 3; - */ - cronSchedule?: string; - - /** - * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 4; - */ - mappings: JobMapping[] = []; - - /** - * @generated from field: mgmt.v1alpha1.JobSource source = 5; - */ - source?: JobSource; - - /** - * @generated from field: repeated mgmt.v1alpha1.CreateJobDestination destinations = 6; - */ - destinations: CreateJobDestination[] = []; - - /** - * Initially trigger a run of this job regardless of its status or cron schedule - * - * @generated from field: bool initiate_job_run = 7; - */ - initiateJobRun = false; - - /** - * Specify timeouts and other workflow options for the underlying temporal workflow - * - * @generated from field: mgmt.v1alpha1.WorkflowOptions workflow_options = 8; - */ - workflowOptions?: WorkflowOptions; - - /** - * Specify timeout and retry options for data synchronization activities - * Data sync activities are any piece of work that involves actually synchronizing data from a source to a destination - * For the data sync and generate jobs, this will be applied per table - * - * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 9; - */ - syncOptions?: ActivityOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "job_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 4, name: "mappings", kind: "message", T: JobMapping, repeated: true }, - { no: 5, name: "source", kind: "message", T: JobSource }, - { no: 6, name: "destinations", kind: "message", T: CreateJobDestination, repeated: true }, - { no: 7, name: "initiate_job_run", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 8, name: "workflow_options", kind: "message", T: WorkflowOptions }, - { no: 9, name: "sync_options", kind: "message", T: ActivityOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRequest { - return new CreateJobRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRequest { - return new CreateJobRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobRequest { - return new CreateJobRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobRequest | PlainMessage | undefined, b: CreateJobRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobRequest, a, b); - } -} - -/** - * Config that contains various timeouts that are configured in the underlying temporal workflow - * More options will come in the future as needed - * - * @generated from message mgmt.v1alpha1.WorkflowOptions - */ -export class WorkflowOptions extends Message { - /** - * The timeout for a single workflow run. - * Measured in seconds - * - * @generated from field: optional int64 run_timeout = 8; - */ - runTimeout?: bigint; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.WorkflowOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 8, name: "run_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): WorkflowOptions { - return new WorkflowOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): WorkflowOptions { - return new WorkflowOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): WorkflowOptions { - return new WorkflowOptions().fromJsonString(jsonString, options); - } - - static equals(a: WorkflowOptions | PlainMessage | undefined, b: WorkflowOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(WorkflowOptions, a, b); - } -} - -/** - * Config that contains various timeouts that are configured in the underlying temporal workflow(s) and activities - * - * @generated from message mgmt.v1alpha1.ActivityOptions - */ -export class ActivityOptions extends Message { - /** - * Total time that a workflow is willing to wait for an activity to complete, including retries. - * Measured in seconds - * - * @generated from field: optional int64 schedule_to_close_timeout = 1; - */ - scheduleToCloseTimeout?: bigint; - - /** - * Max time of a single Temporal Activity execution attempt. - * This timeout should be as short as the longest psosible execution of any activity (e.g. table sync). - * Important to know that this is per retry attempt. Defaults to the schedule to close timeout if not provided. - * Measured in seconds - * - * @generated from field: optional int64 start_to_close_timeout = 2; - */ - startToCloseTimeout?: bigint; - - /** - * Optionally define a retry policy for the activity - * If max attempts is not set, the activity will retry indefinitely until the start to close timeout lapses - * - * @generated from field: mgmt.v1alpha1.RetryPolicy retry_policy = 3; - */ - retryPolicy?: RetryPolicy; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ActivityOptions"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schedule_to_close_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, - { no: 2, name: "start_to_close_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, - { no: 3, name: "retry_policy", kind: "message", T: RetryPolicy }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ActivityOptions { - return new ActivityOptions().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ActivityOptions { - return new ActivityOptions().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ActivityOptions { - return new ActivityOptions().fromJsonString(jsonString, options); - } - - static equals(a: ActivityOptions | PlainMessage | undefined, b: ActivityOptions | PlainMessage | undefined): boolean { - return proto3.util.equals(ActivityOptions, a, b); - } -} - -/** - * Defines the retry policy for an activity - * - * @generated from message mgmt.v1alpha1.RetryPolicy - */ -export class RetryPolicy extends Message { - /** - * Maximum number of attempts. When exceeded the retries stop even if not expired yet. - * If not set or set to 0, it means unlimited, and rely on activity ScheduleToCloseTimeout to stop. - * - * @generated from field: optional int32 maximum_attempts = 1; - */ - maximumAttempts?: number; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RetryPolicy"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "maximum_attempts", kind: "scalar", T: 5 /* ScalarType.INT32 */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RetryPolicy { - return new RetryPolicy().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RetryPolicy { - return new RetryPolicy().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RetryPolicy { - return new RetryPolicy().fromJsonString(jsonString, options); - } - - static equals(a: RetryPolicy | PlainMessage | undefined, b: RetryPolicy | PlainMessage | undefined): boolean { - return proto3.util.equals(RetryPolicy, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobResponse - */ -export class CreateJobResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobResponse { - return new CreateJobResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobResponse { - return new CreateJobResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobResponse { - return new CreateJobResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobResponse | PlainMessage | undefined, b: CreateJobResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobMappingTransformer - */ -export class JobMappingTransformer extends Message { - /** - * @generated from field: mgmt.v1alpha1.TransformerSource source = 1; - */ - source = TransformerSource.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerConfig config = 3; - */ - config?: TransformerConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobMappingTransformer"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, - { no: 3, name: "config", kind: "message", T: TransformerConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobMappingTransformer { - return new JobMappingTransformer().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobMappingTransformer { - return new JobMappingTransformer().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobMappingTransformer { - return new JobMappingTransformer().fromJsonString(jsonString, options); - } - - static equals(a: JobMappingTransformer | PlainMessage | undefined, b: JobMappingTransformer | PlainMessage | undefined): boolean { - return proto3.util.equals(JobMappingTransformer, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobMapping - */ -export class JobMapping extends Message { - /** - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * @generated from field: string table = 2; - */ - table = ""; - - /** - * @generated from field: string column = 3; - */ - column = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobMappingTransformer transformer = 5; - */ - transformer?: JobMappingTransformer; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobMapping"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "transformer", kind: "message", T: JobMappingTransformer }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobMapping { - return new JobMapping().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobMapping { - return new JobMapping().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobMapping { - return new JobMapping().fromJsonString(jsonString, options); - } - - static equals(a: JobMapping | PlainMessage | undefined, b: JobMapping | PlainMessage | undefined): boolean { - return proto3.util.equals(JobMapping, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRequest - */ -export class GetJobRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRequest { - return new GetJobRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRequest { - return new GetJobRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRequest { - return new GetJobRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRequest | PlainMessage | undefined, b: GetJobRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobResponse - */ -export class GetJobResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobResponse { - return new GetJobResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobResponse { - return new GetJobResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobResponse { - return new GetJobResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobResponse | PlainMessage | undefined, b: GetJobResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobScheduleRequest - */ -export class UpdateJobScheduleRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: optional string cron_schedule = 2; - */ - cronSchedule?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobScheduleRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobScheduleRequest { - return new UpdateJobScheduleRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobScheduleRequest { - return new UpdateJobScheduleRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobScheduleRequest { - return new UpdateJobScheduleRequest().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobScheduleRequest | PlainMessage | undefined, b: UpdateJobScheduleRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobScheduleRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobScheduleResponse - */ -export class UpdateJobScheduleResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobScheduleResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobScheduleResponse { - return new UpdateJobScheduleResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobScheduleResponse { - return new UpdateJobScheduleResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobScheduleResponse { - return new UpdateJobScheduleResponse().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobScheduleResponse | PlainMessage | undefined, b: UpdateJobScheduleResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobScheduleResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PauseJobRequest - */ -export class PauseJobRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: bool pause = 2; - */ - pause = false; - - /** - * @generated from field: optional string note = 3; - */ - note?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PauseJobRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "pause", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "note", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PauseJobRequest { - return new PauseJobRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PauseJobRequest { - return new PauseJobRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PauseJobRequest { - return new PauseJobRequest().fromJsonString(jsonString, options); - } - - static equals(a: PauseJobRequest | PlainMessage | undefined, b: PauseJobRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(PauseJobRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PauseJobResponse - */ -export class PauseJobResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PauseJobResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PauseJobResponse { - return new PauseJobResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PauseJobResponse { - return new PauseJobResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PauseJobResponse { - return new PauseJobResponse().fromJsonString(jsonString, options); - } - - static equals(a: PauseJobResponse | PlainMessage | undefined, b: PauseJobResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(PauseJobResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobSourceConnectionRequest - */ -export class UpdateJobSourceConnectionRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobSource source = 2; - */ - source?: JobSource; - - /** - * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 3; - */ - mappings: JobMapping[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobSourceConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "source", kind: "message", T: JobSource }, - { no: 3, name: "mappings", kind: "message", T: JobMapping, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobSourceConnectionRequest { - return new UpdateJobSourceConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobSourceConnectionRequest { - return new UpdateJobSourceConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobSourceConnectionRequest { - return new UpdateJobSourceConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobSourceConnectionRequest | PlainMessage | undefined, b: UpdateJobSourceConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobSourceConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobSourceConnectionResponse - */ -export class UpdateJobSourceConnectionResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobSourceConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobSourceConnectionResponse { - return new UpdateJobSourceConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobSourceConnectionResponse { - return new UpdateJobSourceConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobSourceConnectionResponse { - return new UpdateJobSourceConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobSourceConnectionResponse | PlainMessage | undefined, b: UpdateJobSourceConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobSourceConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PostgresSourceSchemaSubset - */ -export class PostgresSourceSchemaSubset extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.PostgresSourceSchemaOption postgres_schemas = 1; - */ - postgresSchemas: PostgresSourceSchemaOption[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PostgresSourceSchemaSubset"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "postgres_schemas", kind: "message", T: PostgresSourceSchemaOption, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceSchemaSubset { - return new PostgresSourceSchemaSubset().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceSchemaSubset { - return new PostgresSourceSchemaSubset().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PostgresSourceSchemaSubset { - return new PostgresSourceSchemaSubset().fromJsonString(jsonString, options); - } - - static equals(a: PostgresSourceSchemaSubset | PlainMessage | undefined, b: PostgresSourceSchemaSubset | PlainMessage | undefined): boolean { - return proto3.util.equals(PostgresSourceSchemaSubset, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.MysqlSourceSchemaSubset - */ -export class MysqlSourceSchemaSubset extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.MysqlSourceSchemaOption mysql_schemas = 1; - */ - mysqlSchemas: MysqlSourceSchemaOption[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.MysqlSourceSchemaSubset"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "mysql_schemas", kind: "message", T: MysqlSourceSchemaOption, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceSchemaSubset { - return new MysqlSourceSchemaSubset().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceSchemaSubset { - return new MysqlSourceSchemaSubset().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): MysqlSourceSchemaSubset { - return new MysqlSourceSchemaSubset().fromJsonString(jsonString, options); - } - - static equals(a: MysqlSourceSchemaSubset | PlainMessage | undefined, b: MysqlSourceSchemaSubset | PlainMessage | undefined): boolean { - return proto3.util.equals(MysqlSourceSchemaSubset, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobSourceSqlSubetSchemas - */ -export class JobSourceSqlSubetSchemas extends Message { - /** - * @generated from oneof mgmt.v1alpha1.JobSourceSqlSubetSchemas.schemas - */ - schemas: { - /** - * @generated from field: mgmt.v1alpha1.PostgresSourceSchemaSubset postgres_subset = 2; - */ - value: PostgresSourceSchemaSubset; - case: "postgresSubset"; - } | { - /** - * @generated from field: mgmt.v1alpha1.MysqlSourceSchemaSubset mysql_subset = 3; - */ - value: MysqlSourceSchemaSubset; - case: "mysqlSubset"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobSourceSqlSubetSchemas"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 2, name: "postgres_subset", kind: "message", T: PostgresSourceSchemaSubset, oneof: "schemas" }, - { no: 3, name: "mysql_subset", kind: "message", T: MysqlSourceSchemaSubset, oneof: "schemas" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobSourceSqlSubetSchemas { - return new JobSourceSqlSubetSchemas().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobSourceSqlSubetSchemas { - return new JobSourceSqlSubetSchemas().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobSourceSqlSubetSchemas { - return new JobSourceSqlSubetSchemas().fromJsonString(jsonString, options); - } - - static equals(a: JobSourceSqlSubetSchemas | PlainMessage | undefined, b: JobSourceSqlSubetSchemas | PlainMessage | undefined): boolean { - return proto3.util.equals(JobSourceSqlSubetSchemas, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest - */ -export class SetJobSourceSqlConnectionSubsetsRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobSourceSqlSubetSchemas schemas = 2; - */ - schemas?: JobSourceSqlSubetSchemas; - - /** - * @generated from field: bool subset_by_foreign_key_constraints = 3; - */ - subsetByForeignKeyConstraints = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "schemas", kind: "message", T: JobSourceSqlSubetSchemas }, - { no: 3, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { - return new SetJobSourceSqlConnectionSubsetsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { - return new SetJobSourceSqlConnectionSubsetsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { - return new SetJobSourceSqlConnectionSubsetsRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetJobSourceSqlConnectionSubsetsRequest | PlainMessage | undefined, b: SetJobSourceSqlConnectionSubsetsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobSourceSqlConnectionSubsetsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse - */ -export class SetJobSourceSqlConnectionSubsetsResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { - return new SetJobSourceSqlConnectionSubsetsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { - return new SetJobSourceSqlConnectionSubsetsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { - return new SetJobSourceSqlConnectionSubsetsResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetJobSourceSqlConnectionSubsetsResponse | PlainMessage | undefined, b: SetJobSourceSqlConnectionSubsetsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobSourceSqlConnectionSubsetsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobDestinationConnectionRequest - */ -export class UpdateJobDestinationConnectionRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - /** - * @generated from field: string connection_id = 2; - */ - connectionId = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 3; - */ - options?: JobDestinationOptions; - - /** - * @generated from field: string destination_id = 4; - */ - destinationId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "options", kind: "message", T: JobDestinationOptions }, - { no: 4, name: "destination_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobDestinationConnectionRequest { - return new UpdateJobDestinationConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobDestinationConnectionRequest { - return new UpdateJobDestinationConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobDestinationConnectionRequest { - return new UpdateJobDestinationConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobDestinationConnectionRequest | PlainMessage | undefined, b: UpdateJobDestinationConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobDestinationConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateJobDestinationConnectionResponse - */ -export class UpdateJobDestinationConnectionResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobDestinationConnectionResponse { - return new UpdateJobDestinationConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobDestinationConnectionResponse { - return new UpdateJobDestinationConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateJobDestinationConnectionResponse { - return new UpdateJobDestinationConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: UpdateJobDestinationConnectionResponse | PlainMessage | undefined, b: UpdateJobDestinationConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateJobDestinationConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobDestinationConnectionRequest - */ -export class DeleteJobDestinationConnectionRequest extends Message { - /** - * @generated from field: string destination_id = 1; - */ - destinationId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "destination_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobDestinationConnectionRequest { - return new DeleteJobDestinationConnectionRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobDestinationConnectionRequest { - return new DeleteJobDestinationConnectionRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobDestinationConnectionRequest { - return new DeleteJobDestinationConnectionRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobDestinationConnectionRequest | PlainMessage | undefined, b: DeleteJobDestinationConnectionRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobDestinationConnectionRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobDestinationConnectionResponse - */ -export class DeleteJobDestinationConnectionResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobDestinationConnectionResponse { - return new DeleteJobDestinationConnectionResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobDestinationConnectionResponse { - return new DeleteJobDestinationConnectionResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobDestinationConnectionResponse { - return new DeleteJobDestinationConnectionResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobDestinationConnectionResponse | PlainMessage | undefined, b: DeleteJobDestinationConnectionResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobDestinationConnectionResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobDestinationConnectionsRequest - */ -export class CreateJobDestinationConnectionsRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - /** - * @generated from field: repeated mgmt.v1alpha1.CreateJobDestination destinations = 2; - */ - destinations: CreateJobDestination[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "destinations", kind: "message", T: CreateJobDestination, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestinationConnectionsRequest { - return new CreateJobDestinationConnectionsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestinationConnectionsRequest { - return new CreateJobDestinationConnectionsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobDestinationConnectionsRequest { - return new CreateJobDestinationConnectionsRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobDestinationConnectionsRequest | PlainMessage | undefined, b: CreateJobDestinationConnectionsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobDestinationConnectionsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobDestinationConnectionsResponse - */ -export class CreateJobDestinationConnectionsResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestinationConnectionsResponse { - return new CreateJobDestinationConnectionsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestinationConnectionsResponse { - return new CreateJobDestinationConnectionsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobDestinationConnectionsResponse { - return new CreateJobDestinationConnectionsResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobDestinationConnectionsResponse | PlainMessage | undefined, b: CreateJobDestinationConnectionsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobDestinationConnectionsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobRequest - */ -export class DeleteJobRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRequest { - return new DeleteJobRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRequest { - return new DeleteJobRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobRequest { - return new DeleteJobRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobRequest | PlainMessage | undefined, b: DeleteJobRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobResponse - */ -export class DeleteJobResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobResponse { - return new DeleteJobResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobResponse { - return new DeleteJobResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobResponse { - return new DeleteJobResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobResponse | PlainMessage | undefined, b: DeleteJobResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsJobNameAvailableRequest - */ -export class IsJobNameAvailableRequest extends Message { - /** - * @generated from field: string name = 1; - */ - name = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsJobNameAvailableRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsJobNameAvailableRequest { - return new IsJobNameAvailableRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsJobNameAvailableRequest { - return new IsJobNameAvailableRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsJobNameAvailableRequest { - return new IsJobNameAvailableRequest().fromJsonString(jsonString, options); - } - - static equals(a: IsJobNameAvailableRequest | PlainMessage | undefined, b: IsJobNameAvailableRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(IsJobNameAvailableRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsJobNameAvailableResponse - */ -export class IsJobNameAvailableResponse extends Message { - /** - * @generated from field: bool is_available = 1; - */ - isAvailable = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsJobNameAvailableResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsJobNameAvailableResponse { - return new IsJobNameAvailableResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsJobNameAvailableResponse { - return new IsJobNameAvailableResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsJobNameAvailableResponse { - return new IsJobNameAvailableResponse().fromJsonString(jsonString, options); - } - - static equals(a: IsJobNameAvailableResponse | PlainMessage | undefined, b: IsJobNameAvailableResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(IsJobNameAvailableResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunsRequest - */ -export class GetJobRunsRequest extends Message { - /** - * @generated from oneof mgmt.v1alpha1.GetJobRunsRequest.id - */ - id: { - /** - * @generated from field: string job_id = 1; - */ - value: string; - case: "jobId"; - } | { - /** - * @generated from field: string account_id = 2; - */ - value: string; - case: "accountId"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunsRequest { - return new GetJobRunsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunsRequest { - return new GetJobRunsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunsRequest { - return new GetJobRunsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunsRequest | PlainMessage | undefined, b: GetJobRunsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunsResponse - */ -export class GetJobRunsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.JobRun job_runs = 1; - */ - jobRuns: JobRun[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_runs", kind: "message", T: JobRun, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunsResponse { - return new GetJobRunsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunsResponse { - return new GetJobRunsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunsResponse { - return new GetJobRunsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunsResponse | PlainMessage | undefined, b: GetJobRunsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunRequest - */ -export class GetJobRunRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunRequest { - return new GetJobRunRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunRequest { - return new GetJobRunRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunRequest { - return new GetJobRunRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunRequest | PlainMessage | undefined, b: GetJobRunRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunResponse - */ -export class GetJobRunResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.JobRun job_run = 1; - */ - jobRun?: JobRun; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run", kind: "message", T: JobRun }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunResponse { - return new GetJobRunResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunResponse { - return new GetJobRunResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunResponse { - return new GetJobRunResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunResponse | PlainMessage | undefined, b: GetJobRunResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobRunRequest - */ -export class CreateJobRunRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobRunRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRunRequest { - return new CreateJobRunRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRunRequest { - return new CreateJobRunRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobRunRequest { - return new CreateJobRunRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobRunRequest | PlainMessage | undefined, b: CreateJobRunRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobRunRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateJobRunResponse - */ -export class CreateJobRunResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateJobRunResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRunResponse { - return new CreateJobRunResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRunResponse { - return new CreateJobRunResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateJobRunResponse { - return new CreateJobRunResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateJobRunResponse | PlainMessage | undefined, b: CreateJobRunResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateJobRunResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CancelJobRunRequest - */ -export class CancelJobRunRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CancelJobRunRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CancelJobRunRequest { - return new CancelJobRunRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CancelJobRunRequest { - return new CancelJobRunRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CancelJobRunRequest { - return new CancelJobRunRequest().fromJsonString(jsonString, options); - } - - static equals(a: CancelJobRunRequest | PlainMessage | undefined, b: CancelJobRunRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CancelJobRunRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CancelJobRunResponse - */ -export class CancelJobRunResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CancelJobRunResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CancelJobRunResponse { - return new CancelJobRunResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CancelJobRunResponse { - return new CancelJobRunResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CancelJobRunResponse { - return new CancelJobRunResponse().fromJsonString(jsonString, options); - } - - static equals(a: CancelJobRunResponse | PlainMessage | undefined, b: CancelJobRunResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CancelJobRunResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.Job - */ -export class Job extends Message { - /** - * The unique identifier of the job - * - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string created_by_user_id = 2; - */ - createdByUserId = ""; - - /** - * @generated from field: google.protobuf.Timestamp created_at = 3; - */ - createdAt?: Timestamp; - - /** - * @generated from field: string updated_by_user_id = 4; - */ - updatedByUserId = ""; - - /** - * @generated from field: google.protobuf.Timestamp updated_at = 5; - */ - updatedAt?: Timestamp; - - /** - * The unique, friendly name of the job - * - * @generated from field: string name = 6; - */ - name = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobSource source = 7; - */ - source?: JobSource; - - /** - * @generated from field: repeated mgmt.v1alpha1.JobDestination destinations = 8; - */ - destinations: JobDestination[] = []; - - /** - * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 9; - */ - mappings: JobMapping[] = []; - - /** - * @generated from field: optional string cron_schedule = 10; - */ - cronSchedule?: string; - - /** - * The account identifier that a job is associated with - * - * @generated from field: string account_id = 11; - */ - accountId = ""; - - /** - * Specify timeout and retry options for data synchronization activities - * Data sync activities are any piece of work that involves actually synchronizing data from a source to a destination - * For the data sync and generate jobs, this will be applied per table - * - * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 12; - */ - syncOptions?: ActivityOptions; - - /** - * Specify timeouts and other workflow options for the underlying temporal workflow - * - * @generated from field: mgmt.v1alpha1.WorkflowOptions workflow_options = 13; - */ - workflowOptions?: WorkflowOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.Job"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "created_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "created_at", kind: "message", T: Timestamp }, - { no: 4, name: "updated_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "updated_at", kind: "message", T: Timestamp }, - { no: 6, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 7, name: "source", kind: "message", T: JobSource }, - { no: 8, name: "destinations", kind: "message", T: JobDestination, repeated: true }, - { no: 9, name: "mappings", kind: "message", T: JobMapping, repeated: true }, - { no: 10, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - { no: 11, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 12, name: "sync_options", kind: "message", T: ActivityOptions }, - { no: 13, name: "workflow_options", kind: "message", T: WorkflowOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): Job { - return new Job().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): Job { - return new Job().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): Job { - return new Job().fromJsonString(jsonString, options); - } - - static equals(a: Job | PlainMessage | undefined, b: Job | PlainMessage | undefined): boolean { - return proto3.util.equals(Job, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRecentRun - */ -export class JobRecentRun extends Message { - /** - * @generated from field: google.protobuf.Timestamp start_time = 1; - */ - startTime?: Timestamp; - - /** - * @generated from field: string job_run_id = 2; - */ - jobRunId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRecentRun"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "start_time", kind: "message", T: Timestamp }, - { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRecentRun { - return new JobRecentRun().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRecentRun { - return new JobRecentRun().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRecentRun { - return new JobRecentRun().fromJsonString(jsonString, options); - } - - static equals(a: JobRecentRun | PlainMessage | undefined, b: JobRecentRun | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRecentRun, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRecentRunsRequest - */ -export class GetJobRecentRunsRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRecentRunsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRecentRunsRequest { - return new GetJobRecentRunsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRecentRunsRequest { - return new GetJobRecentRunsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRecentRunsRequest { - return new GetJobRecentRunsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRecentRunsRequest | PlainMessage | undefined, b: GetJobRecentRunsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRecentRunsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRecentRunsResponse - */ -export class GetJobRecentRunsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.JobRecentRun recent_runs = 1; - */ - recentRuns: JobRecentRun[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRecentRunsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "recent_runs", kind: "message", T: JobRecentRun, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRecentRunsResponse { - return new GetJobRecentRunsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRecentRunsResponse { - return new GetJobRecentRunsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRecentRunsResponse { - return new GetJobRecentRunsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRecentRunsResponse | PlainMessage | undefined, b: GetJobRecentRunsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRecentRunsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobNextRuns - */ -export class JobNextRuns extends Message { - /** - * @generated from field: repeated google.protobuf.Timestamp next_run_times = 1; - */ - nextRunTimes: Timestamp[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobNextRuns"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "next_run_times", kind: "message", T: Timestamp, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobNextRuns { - return new JobNextRuns().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobNextRuns { - return new JobNextRuns().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobNextRuns { - return new JobNextRuns().fromJsonString(jsonString, options); - } - - static equals(a: JobNextRuns | PlainMessage | undefined, b: JobNextRuns | PlainMessage | undefined): boolean { - return proto3.util.equals(JobNextRuns, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobNextRunsRequest - */ -export class GetJobNextRunsRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobNextRunsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobNextRunsRequest { - return new GetJobNextRunsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobNextRunsRequest { - return new GetJobNextRunsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobNextRunsRequest { - return new GetJobNextRunsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobNextRunsRequest | PlainMessage | undefined, b: GetJobNextRunsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobNextRunsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobNextRunsResponse - */ -export class GetJobNextRunsResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.JobNextRuns next_runs = 1; - */ - nextRuns?: JobNextRuns; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobNextRunsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "next_runs", kind: "message", T: JobNextRuns }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobNextRunsResponse { - return new GetJobNextRunsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobNextRunsResponse { - return new GetJobNextRunsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobNextRunsResponse { - return new GetJobNextRunsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobNextRunsResponse | PlainMessage | undefined, b: GetJobNextRunsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobNextRunsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobStatusRequest - */ -export class GetJobStatusRequest extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobStatusRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusRequest { - return new GetJobStatusRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusRequest { - return new GetJobStatusRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobStatusRequest { - return new GetJobStatusRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobStatusRequest | PlainMessage | undefined, b: GetJobStatusRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobStatusRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobStatusResponse - */ -export class GetJobStatusResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.JobStatus status = 1; - */ - status = JobStatus.UNSPECIFIED; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobStatusResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "status", kind: "enum", T: proto3.getEnumType(JobStatus) }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusResponse { - return new GetJobStatusResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusResponse { - return new GetJobStatusResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobStatusResponse { - return new GetJobStatusResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobStatusResponse | PlainMessage | undefined, b: GetJobStatusResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobStatusResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobStatusRecord - */ -export class JobStatusRecord extends Message { - /** - * @generated from field: string job_id = 1; - */ - jobId = ""; - - /** - * @generated from field: mgmt.v1alpha1.JobStatus status = 2; - */ - status = JobStatus.UNSPECIFIED; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobStatusRecord"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "status", kind: "enum", T: proto3.getEnumType(JobStatus) }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobStatusRecord { - return new JobStatusRecord().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobStatusRecord { - return new JobStatusRecord().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobStatusRecord { - return new JobStatusRecord().fromJsonString(jsonString, options); - } - - static equals(a: JobStatusRecord | PlainMessage | undefined, b: JobStatusRecord | PlainMessage | undefined): boolean { - return proto3.util.equals(JobStatusRecord, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobStatusesRequest - */ -export class GetJobStatusesRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobStatusesRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusesRequest { - return new GetJobStatusesRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusesRequest { - return new GetJobStatusesRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobStatusesRequest { - return new GetJobStatusesRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobStatusesRequest | PlainMessage | undefined, b: GetJobStatusesRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobStatusesRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobStatusesResponse - */ -export class GetJobStatusesResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.JobStatusRecord statuses = 1; - */ - statuses: JobStatusRecord[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobStatusesResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "statuses", kind: "message", T: JobStatusRecord, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusesResponse { - return new GetJobStatusesResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusesResponse { - return new GetJobStatusesResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobStatusesResponse { - return new GetJobStatusesResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobStatusesResponse | PlainMessage | undefined, b: GetJobStatusesResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobStatusesResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ActivityFailure - */ -export class ActivityFailure extends Message { - /** - * @generated from field: string message = 1; - */ - message = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ActivityFailure"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "message", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ActivityFailure { - return new ActivityFailure().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ActivityFailure { - return new ActivityFailure().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ActivityFailure { - return new ActivityFailure().fromJsonString(jsonString, options); - } - - static equals(a: ActivityFailure | PlainMessage | undefined, b: ActivityFailure | PlainMessage | undefined): boolean { - return proto3.util.equals(ActivityFailure, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.PendingActivity - */ -export class PendingActivity extends Message { - /** - * @generated from field: mgmt.v1alpha1.ActivityStatus status = 1; - */ - status = ActivityStatus.UNSPECIFIED; - - /** - * @generated from field: string activity_name = 2; - */ - activityName = ""; - - /** - * @generated from field: optional mgmt.v1alpha1.ActivityFailure last_failure = 3; - */ - lastFailure?: ActivityFailure; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.PendingActivity"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "status", kind: "enum", T: proto3.getEnumType(ActivityStatus) }, - { no: 2, name: "activity_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "last_failure", kind: "message", T: ActivityFailure, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): PendingActivity { - return new PendingActivity().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): PendingActivity { - return new PendingActivity().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): PendingActivity { - return new PendingActivity().fromJsonString(jsonString, options); - } - - static equals(a: PendingActivity | PlainMessage | undefined, b: PendingActivity | PlainMessage | undefined): boolean { - return proto3.util.equals(PendingActivity, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRun - */ -export class JobRun extends Message { - /** - * The id of the job run. This will currently be equivalent to the temporal workflow id - * - * @generated from field: string id = 1; - */ - id = ""; - - /** - * The unique identifier of the job id this run is associated with - * - * @generated from field: string job_id = 2; - */ - jobId = ""; - - /** - * The name of the job run. - * - * @generated from field: string name = 3; - */ - name = ""; - - /** - * the status of the job run - * - * @generated from field: mgmt.v1alpha1.JobRunStatus status = 4; - */ - status = JobRunStatus.UNSPECIFIED; - - /** - * A timestamp of when the run started - * - * @generated from field: google.protobuf.Timestamp started_at = 6; - */ - startedAt?: Timestamp; - - /** - * Available if the run completed or has not yet been archived by the system - * - * @generated from field: optional google.protobuf.Timestamp completed_at = 7; - */ - completedAt?: Timestamp; - - /** - * Pending activities are only returned when retrieving a specific job run and will not be returned when requesting job runs in list format - * - * @generated from field: repeated mgmt.v1alpha1.PendingActivity pending_activities = 8; - */ - pendingActivities: PendingActivity[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRun"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "status", kind: "enum", T: proto3.getEnumType(JobRunStatus) }, - { no: 6, name: "started_at", kind: "message", T: Timestamp }, - { no: 7, name: "completed_at", kind: "message", T: Timestamp, opt: true }, - { no: 8, name: "pending_activities", kind: "message", T: PendingActivity, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRun { - return new JobRun().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRun { - return new JobRun().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRun { - return new JobRun().fromJsonString(jsonString, options); - } - - static equals(a: JobRun | PlainMessage | undefined, b: JobRun | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRun, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRunEventTaskError - */ -export class JobRunEventTaskError extends Message { - /** - * @generated from field: string message = 1; - */ - message = ""; - - /** - * @generated from field: string retry_state = 2; - */ - retryState = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRunEventTaskError"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "message", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "retry_state", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventTaskError { - return new JobRunEventTaskError().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventTaskError { - return new JobRunEventTaskError().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRunEventTaskError { - return new JobRunEventTaskError().fromJsonString(jsonString, options); - } - - static equals(a: JobRunEventTaskError | PlainMessage | undefined, b: JobRunEventTaskError | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRunEventTaskError, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRunEventTask - */ -export class JobRunEventTask extends Message { - /** - * @generated from field: int64 id = 1; - */ - id = protoInt64.zero; - - /** - * @generated from field: string type = 2; - */ - type = ""; - - /** - * @generated from field: google.protobuf.Timestamp event_time = 3; - */ - eventTime?: Timestamp; - - /** - * @generated from field: mgmt.v1alpha1.JobRunEventTaskError error = 4; - */ - error?: JobRunEventTaskError; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRunEventTask"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 2, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "event_time", kind: "message", T: Timestamp }, - { no: 4, name: "error", kind: "message", T: JobRunEventTaskError }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventTask { - return new JobRunEventTask().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventTask { - return new JobRunEventTask().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRunEventTask { - return new JobRunEventTask().fromJsonString(jsonString, options); - } - - static equals(a: JobRunEventTask | PlainMessage | undefined, b: JobRunEventTask | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRunEventTask, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRunSyncMetadata - */ -export class JobRunSyncMetadata extends Message { - /** - * @generated from field: string schema = 1; - */ - schema = ""; - - /** - * @generated from field: string table = 2; - */ - table = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRunSyncMetadata"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRunSyncMetadata { - return new JobRunSyncMetadata().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRunSyncMetadata { - return new JobRunSyncMetadata().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRunSyncMetadata { - return new JobRunSyncMetadata().fromJsonString(jsonString, options); - } - - static equals(a: JobRunSyncMetadata | PlainMessage | undefined, b: JobRunSyncMetadata | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRunSyncMetadata, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRunEventMetadata - */ -export class JobRunEventMetadata extends Message { - /** - * @generated from oneof mgmt.v1alpha1.JobRunEventMetadata.metadata - */ - metadata: { - /** - * @generated from field: mgmt.v1alpha1.JobRunSyncMetadata sync_metadata = 1; - */ - value: JobRunSyncMetadata; - case: "syncMetadata"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRunEventMetadata"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "sync_metadata", kind: "message", T: JobRunSyncMetadata, oneof: "metadata" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventMetadata { - return new JobRunEventMetadata().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventMetadata { - return new JobRunEventMetadata().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRunEventMetadata { - return new JobRunEventMetadata().fromJsonString(jsonString, options); - } - - static equals(a: JobRunEventMetadata | PlainMessage | undefined, b: JobRunEventMetadata | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRunEventMetadata, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.JobRunEvent - */ -export class JobRunEvent extends Message { - /** - * @generated from field: int64 id = 1; - */ - id = protoInt64.zero; - - /** - * @generated from field: string type = 2; - */ - type = ""; - - /** - * @generated from field: google.protobuf.Timestamp start_time = 3; - */ - startTime?: Timestamp; - - /** - * @generated from field: google.protobuf.Timestamp close_time = 4; - */ - closeTime?: Timestamp; - - /** - * @generated from field: mgmt.v1alpha1.JobRunEventMetadata metadata = 5; - */ - metadata?: JobRunEventMetadata; - - /** - * @generated from field: repeated mgmt.v1alpha1.JobRunEventTask tasks = 6; - */ - tasks: JobRunEventTask[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.JobRunEvent"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 2, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "start_time", kind: "message", T: Timestamp }, - { no: 4, name: "close_time", kind: "message", T: Timestamp }, - { no: 5, name: "metadata", kind: "message", T: JobRunEventMetadata }, - { no: 6, name: "tasks", kind: "message", T: JobRunEventTask, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEvent { - return new JobRunEvent().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEvent { - return new JobRunEvent().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): JobRunEvent { - return new JobRunEvent().fromJsonString(jsonString, options); - } - - static equals(a: JobRunEvent | PlainMessage | undefined, b: JobRunEvent | PlainMessage | undefined): boolean { - return proto3.util.equals(JobRunEvent, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunEventsRequest - */ -export class GetJobRunEventsRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunEventsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunEventsRequest { - return new GetJobRunEventsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunEventsRequest { - return new GetJobRunEventsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunEventsRequest { - return new GetJobRunEventsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunEventsRequest | PlainMessage | undefined, b: GetJobRunEventsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunEventsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunEventsResponse - */ -export class GetJobRunEventsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.JobRunEvent events = 1; - */ - events: JobRunEvent[] = []; - - /** - * @generated from field: bool is_run_complete = 2; - */ - isRunComplete = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunEventsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "events", kind: "message", T: JobRunEvent, repeated: true }, - { no: 2, name: "is_run_complete", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunEventsResponse { - return new GetJobRunEventsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunEventsResponse { - return new GetJobRunEventsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunEventsResponse { - return new GetJobRunEventsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunEventsResponse | PlainMessage | undefined, b: GetJobRunEventsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunEventsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobRunRequest - */ -export class DeleteJobRunRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobRunRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRunRequest { - return new DeleteJobRunRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRunRequest { - return new DeleteJobRunRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobRunRequest { - return new DeleteJobRunRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobRunRequest | PlainMessage | undefined, b: DeleteJobRunRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobRunRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteJobRunResponse - */ -export class DeleteJobRunResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteJobRunResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRunResponse { - return new DeleteJobRunResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRunResponse { - return new DeleteJobRunResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteJobRunResponse { - return new DeleteJobRunResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteJobRunResponse | PlainMessage | undefined, b: DeleteJobRunResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteJobRunResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TerminateJobRunRequest - */ -export class TerminateJobRunRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TerminateJobRunRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TerminateJobRunRequest { - return new TerminateJobRunRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TerminateJobRunRequest { - return new TerminateJobRunRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TerminateJobRunRequest { - return new TerminateJobRunRequest().fromJsonString(jsonString, options); - } - - static equals(a: TerminateJobRunRequest | PlainMessage | undefined, b: TerminateJobRunRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(TerminateJobRunRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TerminateJobRunResponse - */ -export class TerminateJobRunResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TerminateJobRunResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TerminateJobRunResponse { - return new TerminateJobRunResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TerminateJobRunResponse { - return new TerminateJobRunResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TerminateJobRunResponse { - return new TerminateJobRunResponse().fromJsonString(jsonString, options); - } - - static equals(a: TerminateJobRunResponse | PlainMessage | undefined, b: TerminateJobRunResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(TerminateJobRunResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunLogsStreamRequest - */ -export class GetJobRunLogsStreamRequest extends Message { - /** - * @generated from field: string job_run_id = 1; - */ - jobRunId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - /** - * The time window in which to retrieve the logs - * - * @generated from field: mgmt.v1alpha1.LogWindow window = 3; - */ - window = LogWindow.NO_TIME_UNSPECIFIED; - - /** - * Whether or not to tail the stream. Note: only works with k8s-pods and is not currently supported with Loki logs - * - * @generated from field: bool should_tail = 4; - */ - shouldTail = false; - - /** - * Optionally provide a max log limit - * - * @generated from field: optional int64 max_log_lines = 5; - */ - maxLogLines?: bigint; - - /** - * Provide a list of log levels to filter by. If any of these are UNSPECIFIED, all log levels are returned. - * - * @generated from field: repeated mgmt.v1alpha1.LogLevel log_levels = 6; - */ - logLevels: LogLevel[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunLogsStreamRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "window", kind: "enum", T: proto3.getEnumType(LogWindow) }, - { no: 4, name: "should_tail", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 5, name: "max_log_lines", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, - { no: 6, name: "log_levels", kind: "enum", T: proto3.getEnumType(LogLevel), repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunLogsStreamRequest { - return new GetJobRunLogsStreamRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunLogsStreamRequest { - return new GetJobRunLogsStreamRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunLogsStreamRequest { - return new GetJobRunLogsStreamRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunLogsStreamRequest | PlainMessage | undefined, b: GetJobRunLogsStreamRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunLogsStreamRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetJobRunLogsStreamResponse - */ -export class GetJobRunLogsStreamResponse extends Message { - /** - * @generated from field: string log_line = 1; - */ - logLine = ""; - - /** - * @generated from field: optional google.protobuf.Timestamp timestamp = 2; - */ - timestamp?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetJobRunLogsStreamResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "log_line", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "timestamp", kind: "message", T: Timestamp, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunLogsStreamResponse { - return new GetJobRunLogsStreamResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunLogsStreamResponse { - return new GetJobRunLogsStreamResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetJobRunLogsStreamResponse { - return new GetJobRunLogsStreamResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetJobRunLogsStreamResponse | PlainMessage | undefined, b: GetJobRunLogsStreamResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetJobRunLogsStreamResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobWorkflowOptionsRequest - */ -export class SetJobWorkflowOptionsRequest extends Message { - /** - * The unique identifier of the job - * - * @generated from field: string id = 1; - */ - id = ""; - - /** - * The workflow options object. The entire object must be provided and will fully overwrite the previous result - * - * @generated from field: mgmt.v1alpha1.WorkflowOptions worfklow_options = 2; - */ - worfklowOptions?: WorkflowOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobWorkflowOptionsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "worfklow_options", kind: "message", T: WorkflowOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobWorkflowOptionsRequest { - return new SetJobWorkflowOptionsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobWorkflowOptionsRequest { - return new SetJobWorkflowOptionsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobWorkflowOptionsRequest { - return new SetJobWorkflowOptionsRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetJobWorkflowOptionsRequest | PlainMessage | undefined, b: SetJobWorkflowOptionsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobWorkflowOptionsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobWorkflowOptionsResponse - */ -export class SetJobWorkflowOptionsResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobWorkflowOptionsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobWorkflowOptionsResponse { - return new SetJobWorkflowOptionsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobWorkflowOptionsResponse { - return new SetJobWorkflowOptionsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobWorkflowOptionsResponse { - return new SetJobWorkflowOptionsResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetJobWorkflowOptionsResponse | PlainMessage | undefined, b: SetJobWorkflowOptionsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobWorkflowOptionsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobSyncOptionsRequest - */ -export class SetJobSyncOptionsRequest extends Message { - /** - * The unique identifier of the job - * - * @generated from field: string id = 1; - */ - id = ""; - - /** - * The sync options object. The entire object must be provided and will fully overwrite the previous result - * - * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 2; - */ - syncOptions?: ActivityOptions; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobSyncOptionsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "sync_options", kind: "message", T: ActivityOptions }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSyncOptionsRequest { - return new SetJobSyncOptionsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSyncOptionsRequest { - return new SetJobSyncOptionsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobSyncOptionsRequest { - return new SetJobSyncOptionsRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetJobSyncOptionsRequest | PlainMessage | undefined, b: SetJobSyncOptionsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobSyncOptionsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetJobSyncOptionsResponse - */ -export class SetJobSyncOptionsResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.Job job = 1; - */ - job?: Job; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetJobSyncOptionsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "job", kind: "message", T: Job }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSyncOptionsResponse { - return new SetJobSyncOptionsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSyncOptionsResponse { - return new SetJobSyncOptionsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetJobSyncOptionsResponse { - return new SetJobSyncOptionsResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetJobSyncOptionsResponse | PlainMessage | undefined, b: SetJobSyncOptionsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetJobSyncOptionsResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts deleted file mode 100644 index 34bc6a22e6..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts +++ /dev/null @@ -1,39 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/metrics.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { GetDailyMetricCountRequest, GetDailyMetricCountResponse, GetMetricCountRequest, GetMetricCountResponse } from "./metrics_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * @generated from service mgmt.v1alpha1.MetricsService - */ -export const MetricsService = { - typeName: "mgmt.v1alpha1.MetricsService", - methods: { - /** - * Retrieve a timed range of records - * - * @generated from rpc mgmt.v1alpha1.MetricsService.GetDailyMetricCount - */ - getDailyMetricCount: { - name: "GetDailyMetricCount", - I: GetDailyMetricCountRequest, - O: GetDailyMetricCountResponse, - kind: MethodKind.Unary, - }, - /** - * For the given metric and time range, returns the total count found - * - * @generated from rpc mgmt.v1alpha1.MetricsService.GetMetricCount - */ - getMetricCount: { - name: "GetMetricCount", - I: GetMetricCountRequest, - O: GetMetricCountResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts deleted file mode 100644 index bb153bf706..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts +++ /dev/null @@ -1,398 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/metrics.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; - -/** - * @generated from enum mgmt.v1alpha1.RangedMetricName - */ -export enum RangedMetricName { - /** - * If unspecified, an error will be thrown - * - * @generated from enum value: RANGED_METRIC_NAME_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * The input_received metric - * - * @generated from enum value: RANGED_METRIC_NAME_INPUT_RECEIVED = 1; - */ - INPUT_RECEIVED = 1, -} -// Retrieve enum metadata with: proto3.getEnumType(RangedMetricName) -proto3.util.setEnumType(RangedMetricName, "mgmt.v1alpha1.RangedMetricName", [ - { no: 0, name: "RANGED_METRIC_NAME_UNSPECIFIED" }, - { no: 1, name: "RANGED_METRIC_NAME_INPUT_RECEIVED" }, -]); - -/** - * Represents a whole or partial calendar date, such as a birthday. The time of - * day and time zone are either specified elsewhere or are insignificant. The - * date is relative to the Gregorian Calendar. This can represent one of the - * following: - * - * * A full date, with non-zero year, month, and day values - * * A month and day value, with a zero year, such as an anniversary - * * A year on its own, with zero month and day values - * * A year and month value, with a zero day, such as a credit card expiration - * date - * - * Related types are [google.type.TimeOfDay][google.type.TimeOfDay] and - * `google.protobuf.Timestamp`. - * - * @generated from message mgmt.v1alpha1.Date - */ -export class Date extends Message { - /** - * Year of the date. Must be from 1 to 9999, or 0 to specify a date without - * a year. - * - * @generated from field: uint32 year = 1; - */ - year = 0; - - /** - * Month of a year. Must be from 1 to 12, or 0 to specify a year without a - * month and day. - * - * @generated from field: uint32 month = 2; - */ - month = 0; - - /** - * Day of a month. Must be from 1 to 31 and valid for the year and month, or 0 - * to specify a year by itself or a year and month where the day isn't - * significant. - * - * @generated from field: uint32 day = 3; - */ - day = 0; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.Date"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "year", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, - { no: 2, name: "month", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, - { no: 3, name: "day", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): Date { - return new Date().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): Date { - return new Date().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): Date { - return new Date().fromJsonString(jsonString, options); - } - - static equals(a: Date | PlainMessage | undefined, b: Date | PlainMessage | undefined): boolean { - return proto3.util.equals(Date, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetDailyMetricCountRequest - */ -export class GetDailyMetricCountRequest extends Message { - /** - * The start day - * - * @generated from field: mgmt.v1alpha1.Date start = 1; - */ - start?: Date; - - /** - * The end day - * - * @generated from field: mgmt.v1alpha1.Date end = 2; - */ - end?: Date; - - /** - * The metric to return - * - * @generated from field: mgmt.v1alpha1.RangedMetricName metric = 3; - */ - metric = RangedMetricName.UNSPECIFIED; - - /** - * @generated from oneof mgmt.v1alpha1.GetDailyMetricCountRequest.identifier - */ - identifier: { - /** - * The account identifier that will be used to filter by - * - * @generated from field: string account_id = 4; - */ - value: string; - case: "accountId"; - } | { - /** - * The job identifier that will be used to filter by - * - * @generated from field: string job_id = 5; - */ - value: string; - case: "jobId"; - } | { - /** - * The run identifier that will be used to filter by - * - * @generated from field: string run_id = 6; - */ - value: string; - case: "runId"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetDailyMetricCountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "start", kind: "message", T: Date }, - { no: 2, name: "end", kind: "message", T: Date }, - { no: 3, name: "metric", kind: "enum", T: proto3.getEnumType(RangedMetricName) }, - { no: 4, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - { no: 5, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - { no: 6, name: "run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetDailyMetricCountRequest { - return new GetDailyMetricCountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetDailyMetricCountRequest { - return new GetDailyMetricCountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetDailyMetricCountRequest { - return new GetDailyMetricCountRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetDailyMetricCountRequest | PlainMessage | undefined, b: GetDailyMetricCountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetDailyMetricCountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetDailyMetricCountResponse - */ -export class GetDailyMetricCountResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.DayResult results = 1; - */ - results: DayResult[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetDailyMetricCountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "results", kind: "message", T: DayResult, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetDailyMetricCountResponse { - return new GetDailyMetricCountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetDailyMetricCountResponse { - return new GetDailyMetricCountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetDailyMetricCountResponse { - return new GetDailyMetricCountResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetDailyMetricCountResponse | PlainMessage | undefined, b: GetDailyMetricCountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetDailyMetricCountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DayResult - */ -export class DayResult extends Message { - /** - * @generated from field: mgmt.v1alpha1.Date date = 1; - */ - date?: Date; - - /** - * @generated from field: uint64 count = 2; - */ - count = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DayResult"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "date", kind: "message", T: Date }, - { no: 2, name: "count", kind: "scalar", T: 4 /* ScalarType.UINT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DayResult { - return new DayResult().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DayResult { - return new DayResult().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DayResult { - return new DayResult().fromJsonString(jsonString, options); - } - - static equals(a: DayResult | PlainMessage | undefined, b: DayResult | PlainMessage | undefined): boolean { - return proto3.util.equals(DayResult, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetMetricCountRequest - */ -export class GetMetricCountRequest extends Message { - /** - * The start time - * - * @generated from field: google.protobuf.Timestamp start = 1; - */ - start?: Timestamp; - - /** - * The end time - * - * @generated from field: google.protobuf.Timestamp end = 2; - */ - end?: Timestamp; - - /** - * The metric to return - * - * @generated from field: mgmt.v1alpha1.RangedMetricName metric = 3; - */ - metric = RangedMetricName.UNSPECIFIED; - - /** - * @generated from oneof mgmt.v1alpha1.GetMetricCountRequest.identifier - */ - identifier: { - /** - * The account identifier that will be used to filter by - * - * @generated from field: string account_id = 4; - */ - value: string; - case: "accountId"; - } | { - /** - * The job identifier that will be used to filter by - * - * @generated from field: string job_id = 5; - */ - value: string; - case: "jobId"; - } | { - /** - * The run identifier that will be used to filter by - * - * @generated from field: string run_id = 6; - */ - value: string; - case: "runId"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetMetricCountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "start", kind: "message", T: Timestamp }, - { no: 2, name: "end", kind: "message", T: Timestamp }, - { no: 3, name: "metric", kind: "enum", T: proto3.getEnumType(RangedMetricName) }, - { no: 4, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - { no: 5, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - { no: 6, name: "run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetMetricCountRequest { - return new GetMetricCountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetMetricCountRequest { - return new GetMetricCountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetMetricCountRequest { - return new GetMetricCountRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetMetricCountRequest | PlainMessage | undefined, b: GetMetricCountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetMetricCountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetMetricCountResponse - */ -export class GetMetricCountResponse extends Message { - /** - * The summed up count of the metric based on the input query and timerange specified - * - * @generated from field: uint64 count = 1; - */ - count = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetMetricCountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "count", kind: "scalar", T: 4 /* ScalarType.UINT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetMetricCountResponse { - return new GetMetricCountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetMetricCountResponse { - return new GetMetricCountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetMetricCountResponse { - return new GetMetricCountResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetMetricCountResponse | PlainMessage | undefined, b: GetMetricCountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetMetricCountResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts deleted file mode 100644 index 9ecd9ab022..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts +++ /dev/null @@ -1,107 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/transformer.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { CreateUserDefinedTransformerRequest, CreateUserDefinedTransformerResponse, DeleteUserDefinedTransformerRequest, DeleteUserDefinedTransformerResponse, GetSystemTransformerBySourceRequest, GetSystemTransformerBySourceResponse, GetSystemTransformersRequest, GetSystemTransformersResponse, GetUserDefinedTransformerByIdRequest, GetUserDefinedTransformerByIdResponse, GetUserDefinedTransformersRequest, GetUserDefinedTransformersResponse, IsTransformerNameAvailableRequest, IsTransformerNameAvailableResponse, UpdateUserDefinedTransformerRequest, UpdateUserDefinedTransformerResponse, ValidateUserJavascriptCodeRequest, ValidateUserJavascriptCodeResponse, ValidateUserRegexCodeRequest, ValidateUserRegexCodeResponse } from "./transformer_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * @generated from service mgmt.v1alpha1.TransformersService - */ -export const TransformersService = { - typeName: "mgmt.v1alpha1.TransformersService", - methods: { - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.GetSystemTransformers - */ - getSystemTransformers: { - name: "GetSystemTransformers", - I: GetSystemTransformersRequest, - O: GetSystemTransformersResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.GetSystemTransformerBySource - */ - getSystemTransformerBySource: { - name: "GetSystemTransformerBySource", - I: GetSystemTransformerBySourceRequest, - O: GetSystemTransformerBySourceResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.GetUserDefinedTransformers - */ - getUserDefinedTransformers: { - name: "GetUserDefinedTransformers", - I: GetUserDefinedTransformersRequest, - O: GetUserDefinedTransformersResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.GetUserDefinedTransformerById - */ - getUserDefinedTransformerById: { - name: "GetUserDefinedTransformerById", - I: GetUserDefinedTransformerByIdRequest, - O: GetUserDefinedTransformerByIdResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.CreateUserDefinedTransformer - */ - createUserDefinedTransformer: { - name: "CreateUserDefinedTransformer", - I: CreateUserDefinedTransformerRequest, - O: CreateUserDefinedTransformerResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.DeleteUserDefinedTransformer - */ - deleteUserDefinedTransformer: { - name: "DeleteUserDefinedTransformer", - I: DeleteUserDefinedTransformerRequest, - O: DeleteUserDefinedTransformerResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.UpdateUserDefinedTransformer - */ - updateUserDefinedTransformer: { - name: "UpdateUserDefinedTransformer", - I: UpdateUserDefinedTransformerRequest, - O: UpdateUserDefinedTransformerResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.IsTransformerNameAvailable - */ - isTransformerNameAvailable: { - name: "IsTransformerNameAvailable", - I: IsTransformerNameAvailableRequest, - O: IsTransformerNameAvailableResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.ValidateUserJavascriptCode - */ - validateUserJavascriptCode: { - name: "ValidateUserJavascriptCode", - I: ValidateUserJavascriptCodeRequest, - O: ValidateUserJavascriptCodeResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.TransformersService.ValidateUserRegexCode - */ - validateUserRegexCode: { - name: "ValidateUserRegexCode", - I: ValidateUserRegexCodeRequest, - O: ValidateUserRegexCodeResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts deleted file mode 100644 index 206238d0b0..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts +++ /dev/null @@ -1,3180 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/transformer.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; - -/** - * @generated from enum mgmt.v1alpha1.TransformerSource - */ -export enum TransformerSource { - /** - * @generated from enum value: TRANSFORMER_SOURCE_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_PASSTHROUGH = 1; - */ - PASSTHROUGH = 1, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_DEFAULT = 2; - */ - GENERATE_DEFAULT = 2, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT = 3; - */ - TRANSFORM_JAVASCRIPT = 3, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_EMAIL = 4; - */ - GENERATE_EMAIL = 4, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_EMAIL = 5; - */ - TRANSFORM_EMAIL = 5, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_BOOL = 6; - */ - GENERATE_BOOL = 6, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER = 7; - */ - GENERATE_CARD_NUMBER = 7, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CITY = 8; - */ - GENERATE_CITY = 8, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER = 9; - */ - GENERATE_E164_PHONE_NUMBER = 9, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FIRST_NAME = 10; - */ - GENERATE_FIRST_NAME = 10, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FLOAT64 = 11; - */ - GENERATE_FLOAT64 = 11, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS = 12; - */ - GENERATE_FULL_ADDRESS = 12, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FULL_NAME = 13; - */ - GENERATE_FULL_NAME = 13, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_GENDER = 14; - */ - GENERATE_GENDER = 14, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER = 15; - */ - GENERATE_INT64_PHONE_NUMBER = 15, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_INT64 = 16; - */ - GENERATE_INT64 = 16, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64 = 17; - */ - GENERATE_RANDOM_INT64 = 17, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_LAST_NAME = 18; - */ - GENERATE_LAST_NAME = 18, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_SHA256HASH = 19; - */ - GENERATE_SHA256HASH = 19, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_SSN = 20; - */ - GENERATE_SSN = 20, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STATE = 21; - */ - GENERATE_STATE = 21, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS = 22; - */ - GENERATE_STREET_ADDRESS = 22, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER = 23; - */ - GENERATE_STRING_PHONE_NUMBER = 23, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STRING = 24; - */ - GENERATE_STRING = 24, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING = 25; - */ - GENERATE_RANDOM_STRING = 25, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP = 26; - */ - GENERATE_UNIXTIMESTAMP = 26, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_USERNAME = 27; - */ - GENERATE_USERNAME = 27, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP = 28; - */ - GENERATE_UTCTIMESTAMP = 28, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UUID = 29; - */ - GENERATE_UUID = 29, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_ZIPCODE = 30; - */ - GENERATE_ZIPCODE = 30, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER = 31; - */ - TRANSFORM_E164_PHONE_NUMBER = 31, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME = 32; - */ - TRANSFORM_FIRST_NAME = 32, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FLOAT64 = 33; - */ - TRANSFORM_FLOAT64 = 33, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME = 34; - */ - TRANSFORM_FULL_NAME = 34, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER = 35; - */ - TRANSFORM_INT64_PHONE_NUMBER = 35, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_INT64 = 36; - */ - TRANSFORM_INT64 = 36, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME = 37; - */ - TRANSFORM_LAST_NAME = 37, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER = 38; - */ - TRANSFORM_PHONE_NUMBER = 38, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_STRING = 39; - */ - TRANSFORM_STRING = 39, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_NULL = 40; - */ - GENERATE_NULL = 40, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CATEGORICAL = 42; - */ - GENERATE_CATEGORICAL = 42, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE = 43; - */ - TRANSFORM_CHARACTER_SCRAMBLE = 43, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_USER_DEFINED = 44; - */ - USER_DEFINED = 44, - - /** - * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT = 45; - */ - GENERATE_JAVASCRIPT = 45, -} -// Retrieve enum metadata with: proto3.getEnumType(TransformerSource) -proto3.util.setEnumType(TransformerSource, "mgmt.v1alpha1.TransformerSource", [ - { no: 0, name: "TRANSFORMER_SOURCE_UNSPECIFIED" }, - { no: 1, name: "TRANSFORMER_SOURCE_PASSTHROUGH" }, - { no: 2, name: "TRANSFORMER_SOURCE_GENERATE_DEFAULT" }, - { no: 3, name: "TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT" }, - { no: 4, name: "TRANSFORMER_SOURCE_GENERATE_EMAIL" }, - { no: 5, name: "TRANSFORMER_SOURCE_TRANSFORM_EMAIL" }, - { no: 6, name: "TRANSFORMER_SOURCE_GENERATE_BOOL" }, - { no: 7, name: "TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER" }, - { no: 8, name: "TRANSFORMER_SOURCE_GENERATE_CITY" }, - { no: 9, name: "TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER" }, - { no: 10, name: "TRANSFORMER_SOURCE_GENERATE_FIRST_NAME" }, - { no: 11, name: "TRANSFORMER_SOURCE_GENERATE_FLOAT64" }, - { no: 12, name: "TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS" }, - { no: 13, name: "TRANSFORMER_SOURCE_GENERATE_FULL_NAME" }, - { no: 14, name: "TRANSFORMER_SOURCE_GENERATE_GENDER" }, - { no: 15, name: "TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER" }, - { no: 16, name: "TRANSFORMER_SOURCE_GENERATE_INT64" }, - { no: 17, name: "TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64" }, - { no: 18, name: "TRANSFORMER_SOURCE_GENERATE_LAST_NAME" }, - { no: 19, name: "TRANSFORMER_SOURCE_GENERATE_SHA256HASH" }, - { no: 20, name: "TRANSFORMER_SOURCE_GENERATE_SSN" }, - { no: 21, name: "TRANSFORMER_SOURCE_GENERATE_STATE" }, - { no: 22, name: "TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS" }, - { no: 23, name: "TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER" }, - { no: 24, name: "TRANSFORMER_SOURCE_GENERATE_STRING" }, - { no: 25, name: "TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING" }, - { no: 26, name: "TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP" }, - { no: 27, name: "TRANSFORMER_SOURCE_GENERATE_USERNAME" }, - { no: 28, name: "TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP" }, - { no: 29, name: "TRANSFORMER_SOURCE_GENERATE_UUID" }, - { no: 30, name: "TRANSFORMER_SOURCE_GENERATE_ZIPCODE" }, - { no: 31, name: "TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER" }, - { no: 32, name: "TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME" }, - { no: 33, name: "TRANSFORMER_SOURCE_TRANSFORM_FLOAT64" }, - { no: 34, name: "TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME" }, - { no: 35, name: "TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER" }, - { no: 36, name: "TRANSFORMER_SOURCE_TRANSFORM_INT64" }, - { no: 37, name: "TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME" }, - { no: 38, name: "TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER" }, - { no: 39, name: "TRANSFORMER_SOURCE_TRANSFORM_STRING" }, - { no: 40, name: "TRANSFORMER_SOURCE_GENERATE_NULL" }, - { no: 42, name: "TRANSFORMER_SOURCE_GENERATE_CATEGORICAL" }, - { no: 43, name: "TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE" }, - { no: 44, name: "TRANSFORMER_SOURCE_USER_DEFINED" }, - { no: 45, name: "TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT" }, -]); - -/** - * @generated from enum mgmt.v1alpha1.TransformerDataType - */ -export enum TransformerDataType { - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_STRING = 1; - */ - STRING = 1, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_INT64 = 2; - */ - INT64 = 2, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_BOOLEAN = 3; - */ - BOOLEAN = 3, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_FLOAT64 = 4; - */ - FLOAT64 = 4, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_NULL = 5; - */ - NULL = 5, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_ANY = 6; - */ - ANY = 6, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_TIME = 7; - */ - TIME = 7, - - /** - * @generated from enum value: TRANSFORMER_DATA_TYPE_UUID = 8; - */ - UUID = 8, -} -// Retrieve enum metadata with: proto3.getEnumType(TransformerDataType) -proto3.util.setEnumType(TransformerDataType, "mgmt.v1alpha1.TransformerDataType", [ - { no: 0, name: "TRANSFORMER_DATA_TYPE_UNSPECIFIED" }, - { no: 1, name: "TRANSFORMER_DATA_TYPE_STRING" }, - { no: 2, name: "TRANSFORMER_DATA_TYPE_INT64" }, - { no: 3, name: "TRANSFORMER_DATA_TYPE_BOOLEAN" }, - { no: 4, name: "TRANSFORMER_DATA_TYPE_FLOAT64" }, - { no: 5, name: "TRANSFORMER_DATA_TYPE_NULL" }, - { no: 6, name: "TRANSFORMER_DATA_TYPE_ANY" }, - { no: 7, name: "TRANSFORMER_DATA_TYPE_TIME" }, - { no: 8, name: "TRANSFORMER_DATA_TYPE_UUID" }, -]); - -/** - * @generated from enum mgmt.v1alpha1.SupportedJobType - */ -export enum SupportedJobType { - /** - * @generated from enum value: SUPPORTED_JOB_TYPE_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: SUPPORTED_JOB_TYPE_SYNC = 1; - */ - SYNC = 1, - - /** - * @generated from enum value: SUPPORTED_JOB_TYPE_GENERATE = 2; - */ - GENERATE = 2, -} -// Retrieve enum metadata with: proto3.getEnumType(SupportedJobType) -proto3.util.setEnumType(SupportedJobType, "mgmt.v1alpha1.SupportedJobType", [ - { no: 0, name: "SUPPORTED_JOB_TYPE_UNSPECIFIED" }, - { no: 1, name: "SUPPORTED_JOB_TYPE_SYNC" }, - { no: 2, name: "SUPPORTED_JOB_TYPE_GENERATE" }, -]); - -/** - * @generated from message mgmt.v1alpha1.GetSystemTransformersRequest - */ -export class GetSystemTransformersRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemTransformersRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformersRequest { - return new GetSystemTransformersRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformersRequest { - return new GetSystemTransformersRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformersRequest { - return new GetSystemTransformersRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemTransformersRequest | PlainMessage | undefined, b: GetSystemTransformersRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemTransformersRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetSystemTransformersResponse - */ -export class GetSystemTransformersResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.SystemTransformer transformers = 1; - */ - transformers: SystemTransformer[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemTransformersResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformers", kind: "message", T: SystemTransformer, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformersResponse { - return new GetSystemTransformersResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformersResponse { - return new GetSystemTransformersResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformersResponse { - return new GetSystemTransformersResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemTransformersResponse | PlainMessage | undefined, b: GetSystemTransformersResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemTransformersResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetSystemTransformerBySourceRequest - */ -export class GetSystemTransformerBySourceRequest extends Message { - /** - * @generated from field: mgmt.v1alpha1.TransformerSource source = 1; - */ - source = TransformerSource.UNSPECIFIED; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemTransformerBySourceRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformerBySourceRequest { - return new GetSystemTransformerBySourceRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformerBySourceRequest { - return new GetSystemTransformerBySourceRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformerBySourceRequest { - return new GetSystemTransformerBySourceRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemTransformerBySourceRequest | PlainMessage | undefined, b: GetSystemTransformerBySourceRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemTransformerBySourceRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetSystemTransformerBySourceResponse - */ -export class GetSystemTransformerBySourceResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.SystemTransformer transformer = 1; - */ - transformer?: SystemTransformer; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemTransformerBySourceResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer", kind: "message", T: SystemTransformer }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformerBySourceResponse { - return new GetSystemTransformerBySourceResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformerBySourceResponse { - return new GetSystemTransformerBySourceResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformerBySourceResponse { - return new GetSystemTransformerBySourceResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemTransformerBySourceResponse | PlainMessage | undefined, b: GetSystemTransformerBySourceResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemTransformerBySourceResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserDefinedTransformersRequest - */ -export class GetUserDefinedTransformersRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformersRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformersRequest { - return new GetUserDefinedTransformersRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformersRequest { - return new GetUserDefinedTransformersRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformersRequest { - return new GetUserDefinedTransformersRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetUserDefinedTransformersRequest | PlainMessage | undefined, b: GetUserDefinedTransformersRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserDefinedTransformersRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserDefinedTransformersResponse - */ -export class GetUserDefinedTransformersResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.UserDefinedTransformer transformers = 1; - */ - transformers: UserDefinedTransformer[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformersResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformers", kind: "message", T: UserDefinedTransformer, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformersResponse { - return new GetUserDefinedTransformersResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformersResponse { - return new GetUserDefinedTransformersResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformersResponse { - return new GetUserDefinedTransformersResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetUserDefinedTransformersResponse | PlainMessage | undefined, b: GetUserDefinedTransformersResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserDefinedTransformersResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest - */ -export class GetUserDefinedTransformerByIdRequest extends Message { - /** - * @generated from field: string transformer_id = 1; - */ - transformerId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformerByIdRequest { - return new GetUserDefinedTransformerByIdRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformerByIdRequest { - return new GetUserDefinedTransformerByIdRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformerByIdRequest { - return new GetUserDefinedTransformerByIdRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetUserDefinedTransformerByIdRequest | PlainMessage | undefined, b: GetUserDefinedTransformerByIdRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserDefinedTransformerByIdRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse - */ -export class GetUserDefinedTransformerByIdResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; - */ - transformer?: UserDefinedTransformer; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformerByIdResponse { - return new GetUserDefinedTransformerByIdResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformerByIdResponse { - return new GetUserDefinedTransformerByIdResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformerByIdResponse { - return new GetUserDefinedTransformerByIdResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetUserDefinedTransformerByIdResponse | PlainMessage | undefined, b: GetUserDefinedTransformerByIdResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserDefinedTransformerByIdResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateUserDefinedTransformerRequest - */ -export class CreateUserDefinedTransformerRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: string description = 3; - */ - description = ""; - - /** - * @deprecated - * - * @generated from field: string type = 4; - */ - type = ""; - - /** - * @generated from field: mgmt.v1alpha1.TransformerSource source = 5; - */ - source = TransformerSource.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerConfig transformer_config = 6; - */ - transformerConfig?: TransformerConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateUserDefinedTransformerRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, - { no: 6, name: "transformer_config", kind: "message", T: TransformerConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateUserDefinedTransformerRequest { - return new CreateUserDefinedTransformerRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateUserDefinedTransformerRequest { - return new CreateUserDefinedTransformerRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateUserDefinedTransformerRequest { - return new CreateUserDefinedTransformerRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateUserDefinedTransformerRequest | PlainMessage | undefined, b: CreateUserDefinedTransformerRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateUserDefinedTransformerRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateUserDefinedTransformerResponse - */ -export class CreateUserDefinedTransformerResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; - */ - transformer?: UserDefinedTransformer; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateUserDefinedTransformerResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateUserDefinedTransformerResponse { - return new CreateUserDefinedTransformerResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateUserDefinedTransformerResponse { - return new CreateUserDefinedTransformerResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateUserDefinedTransformerResponse { - return new CreateUserDefinedTransformerResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateUserDefinedTransformerResponse | PlainMessage | undefined, b: CreateUserDefinedTransformerResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateUserDefinedTransformerResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteUserDefinedTransformerRequest - */ -export class DeleteUserDefinedTransformerRequest extends Message { - /** - * @generated from field: string transformer_id = 1; - */ - transformerId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteUserDefinedTransformerRequest { - return new DeleteUserDefinedTransformerRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteUserDefinedTransformerRequest { - return new DeleteUserDefinedTransformerRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteUserDefinedTransformerRequest { - return new DeleteUserDefinedTransformerRequest().fromJsonString(jsonString, options); - } - - static equals(a: DeleteUserDefinedTransformerRequest | PlainMessage | undefined, b: DeleteUserDefinedTransformerRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteUserDefinedTransformerRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.DeleteUserDefinedTransformerResponse - */ -export class DeleteUserDefinedTransformerResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): DeleteUserDefinedTransformerResponse { - return new DeleteUserDefinedTransformerResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): DeleteUserDefinedTransformerResponse { - return new DeleteUserDefinedTransformerResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): DeleteUserDefinedTransformerResponse { - return new DeleteUserDefinedTransformerResponse().fromJsonString(jsonString, options); - } - - static equals(a: DeleteUserDefinedTransformerResponse | PlainMessage | undefined, b: DeleteUserDefinedTransformerResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(DeleteUserDefinedTransformerResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateUserDefinedTransformerRequest - */ -export class UpdateUserDefinedTransformerRequest extends Message { - /** - * @generated from field: string transformer_id = 1; - */ - transformerId = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: string description = 3; - */ - description = ""; - - /** - * @generated from field: mgmt.v1alpha1.TransformerConfig transformer_config = 4; - */ - transformerConfig?: TransformerConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "transformer_config", kind: "message", T: TransformerConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateUserDefinedTransformerRequest { - return new UpdateUserDefinedTransformerRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateUserDefinedTransformerRequest { - return new UpdateUserDefinedTransformerRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateUserDefinedTransformerRequest { - return new UpdateUserDefinedTransformerRequest().fromJsonString(jsonString, options); - } - - static equals(a: UpdateUserDefinedTransformerRequest | PlainMessage | undefined, b: UpdateUserDefinedTransformerRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateUserDefinedTransformerRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UpdateUserDefinedTransformerResponse - */ -export class UpdateUserDefinedTransformerResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; - */ - transformer?: UserDefinedTransformer; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UpdateUserDefinedTransformerResponse { - return new UpdateUserDefinedTransformerResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UpdateUserDefinedTransformerResponse { - return new UpdateUserDefinedTransformerResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UpdateUserDefinedTransformerResponse { - return new UpdateUserDefinedTransformerResponse().fromJsonString(jsonString, options); - } - - static equals(a: UpdateUserDefinedTransformerResponse | PlainMessage | undefined, b: UpdateUserDefinedTransformerResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(UpdateUserDefinedTransformerResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsTransformerNameAvailableRequest - */ -export class IsTransformerNameAvailableRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string transformer_name = 2; - */ - transformerName = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsTransformerNameAvailableRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "transformer_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsTransformerNameAvailableRequest { - return new IsTransformerNameAvailableRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsTransformerNameAvailableRequest { - return new IsTransformerNameAvailableRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsTransformerNameAvailableRequest { - return new IsTransformerNameAvailableRequest().fromJsonString(jsonString, options); - } - - static equals(a: IsTransformerNameAvailableRequest | PlainMessage | undefined, b: IsTransformerNameAvailableRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(IsTransformerNameAvailableRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsTransformerNameAvailableResponse - */ -export class IsTransformerNameAvailableResponse extends Message { - /** - * @generated from field: bool is_available = 1; - */ - isAvailable = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsTransformerNameAvailableResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsTransformerNameAvailableResponse { - return new IsTransformerNameAvailableResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsTransformerNameAvailableResponse { - return new IsTransformerNameAvailableResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsTransformerNameAvailableResponse { - return new IsTransformerNameAvailableResponse().fromJsonString(jsonString, options); - } - - static equals(a: IsTransformerNameAvailableResponse | PlainMessage | undefined, b: IsTransformerNameAvailableResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(IsTransformerNameAvailableResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UserDefinedTransformer - */ -export class UserDefinedTransformer extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: string description = 3; - */ - description = ""; - - /** - * This property is readonly and is calculated based off the origin system transformer - * - * @generated from field: mgmt.v1alpha1.TransformerDataType data_type = 5; - */ - dataType = TransformerDataType.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerSource source = 6; - */ - source = TransformerSource.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerConfig config = 7; - */ - config?: TransformerConfig; - - /** - * @generated from field: google.protobuf.Timestamp created_at = 8; - */ - createdAt?: Timestamp; - - /** - * @generated from field: google.protobuf.Timestamp updated_at = 9; - */ - updatedAt?: Timestamp; - - /** - * @generated from field: string account_id = 10; - */ - accountId = ""; - - /** - * @generated from field: repeated mgmt.v1alpha1.TransformerDataType data_types = 11; - */ - dataTypes: TransformerDataType[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UserDefinedTransformer"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "data_type", kind: "enum", T: proto3.getEnumType(TransformerDataType) }, - { no: 6, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, - { no: 7, name: "config", kind: "message", T: TransformerConfig }, - { no: 8, name: "created_at", kind: "message", T: Timestamp }, - { no: 9, name: "updated_at", kind: "message", T: Timestamp }, - { no: 10, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 11, name: "data_types", kind: "enum", T: proto3.getEnumType(TransformerDataType), repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UserDefinedTransformer { - return new UserDefinedTransformer().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UserDefinedTransformer { - return new UserDefinedTransformer().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UserDefinedTransformer { - return new UserDefinedTransformer().fromJsonString(jsonString, options); - } - - static equals(a: UserDefinedTransformer | PlainMessage | undefined, b: UserDefinedTransformer | PlainMessage | undefined): boolean { - return proto3.util.equals(UserDefinedTransformer, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SystemTransformer - */ -export class SystemTransformer extends Message { - /** - * @generated from field: string name = 1; - */ - name = ""; - - /** - * @generated from field: string description = 2; - */ - description = ""; - - /** - * @generated from field: mgmt.v1alpha1.TransformerDataType data_type = 3; - */ - dataType = TransformerDataType.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerSource source = 4; - */ - source = TransformerSource.UNSPECIFIED; - - /** - * @generated from field: mgmt.v1alpha1.TransformerConfig config = 5; - */ - config?: TransformerConfig; - - /** - * @generated from field: repeated mgmt.v1alpha1.TransformerDataType data_types = 6; - */ - dataTypes: TransformerDataType[] = []; - - /** - * @generated from field: repeated mgmt.v1alpha1.SupportedJobType supported_job_types = 7; - */ - supportedJobTypes: SupportedJobType[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SystemTransformer"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "data_type", kind: "enum", T: proto3.getEnumType(TransformerDataType) }, - { no: 4, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, - { no: 5, name: "config", kind: "message", T: TransformerConfig }, - { no: 6, name: "data_types", kind: "enum", T: proto3.getEnumType(TransformerDataType), repeated: true }, - { no: 7, name: "supported_job_types", kind: "enum", T: proto3.getEnumType(SupportedJobType), repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SystemTransformer { - return new SystemTransformer().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SystemTransformer { - return new SystemTransformer().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SystemTransformer { - return new SystemTransformer().fromJsonString(jsonString, options); - } - - static equals(a: SystemTransformer | PlainMessage | undefined, b: SystemTransformer | PlainMessage | undefined): boolean { - return proto3.util.equals(SystemTransformer, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformerConfig - */ -export class TransformerConfig extends Message { - /** - * @generated from oneof mgmt.v1alpha1.TransformerConfig.config - */ - config: { - /** - * @generated from field: mgmt.v1alpha1.GenerateEmail generate_email_config = 1; - */ - value: GenerateEmail; - case: "generateEmailConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformEmail transform_email_config = 2; - */ - value: TransformEmail; - case: "transformEmailConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateBool generate_bool_config = 3; - */ - value: GenerateBool; - case: "generateBoolConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateCardNumber generate_card_number_config = 4; - */ - value: GenerateCardNumber; - case: "generateCardNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateCity generate_city_config = 5; - */ - value: GenerateCity; - case: "generateCityConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateE164PhoneNumber generate_e164_phone_number_config = 6; - */ - value: GenerateE164PhoneNumber; - case: "generateE164PhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateFirstName generate_first_name_config = 7; - */ - value: GenerateFirstName; - case: "generateFirstNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateFloat64 generate_float64_config = 8; - */ - value: GenerateFloat64; - case: "generateFloat64Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateFullAddress generate_full_address_config = 9; - */ - value: GenerateFullAddress; - case: "generateFullAddressConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateFullName generate_full_name_config = 10; - */ - value: GenerateFullName; - case: "generateFullNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateGender generate_gender_config = 11; - */ - value: GenerateGender; - case: "generateGenderConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateInt64PhoneNumber generate_int64_phone_number_config = 12; - */ - value: GenerateInt64PhoneNumber; - case: "generateInt64PhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateInt64 generate_int64_config = 13; - */ - value: GenerateInt64; - case: "generateInt64Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateLastName generate_last_name_config = 14; - */ - value: GenerateLastName; - case: "generateLastNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateSha256Hash generate_sha256hash_config = 15; - */ - value: GenerateSha256Hash; - case: "generateSha256hashConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateSSN generate_ssn_config = 16; - */ - value: GenerateSSN; - case: "generateSsnConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateState generate_state_config = 17; - */ - value: GenerateState; - case: "generateStateConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateStreetAddress generate_street_address_config = 18; - */ - value: GenerateStreetAddress; - case: "generateStreetAddressConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateStringPhoneNumber generate_string_phone_number_config = 19; - */ - value: GenerateStringPhoneNumber; - case: "generateStringPhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateString generate_string_config = 20; - */ - value: GenerateString; - case: "generateStringConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateUnixTimestamp generate_unixtimestamp_config = 21; - */ - value: GenerateUnixTimestamp; - case: "generateUnixtimestampConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateUsername generate_username_config = 22; - */ - value: GenerateUsername; - case: "generateUsernameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateUtcTimestamp generate_utctimestamp_config = 23; - */ - value: GenerateUtcTimestamp; - case: "generateUtctimestampConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateUuid generate_uuid_config = 24; - */ - value: GenerateUuid; - case: "generateUuidConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateZipcode generate_zipcode_config = 25; - */ - value: GenerateZipcode; - case: "generateZipcodeConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformE164PhoneNumber transform_e164_phone_number_config = 26; - */ - value: TransformE164PhoneNumber; - case: "transformE164PhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformFirstName transform_first_name_config = 27; - */ - value: TransformFirstName; - case: "transformFirstNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformFloat64 transform_float64_config = 28; - */ - value: TransformFloat64; - case: "transformFloat64Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformFullName transform_full_name_config = 29; - */ - value: TransformFullName; - case: "transformFullNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformInt64PhoneNumber transform_int64_phone_number_config = 30; - */ - value: TransformInt64PhoneNumber; - case: "transformInt64PhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformInt64 transform_int64_config = 31; - */ - value: TransformInt64; - case: "transformInt64Config"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformLastName transform_last_name_config = 32; - */ - value: TransformLastName; - case: "transformLastNameConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformPhoneNumber transform_phone_number_config = 33; - */ - value: TransformPhoneNumber; - case: "transformPhoneNumberConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformString transform_string_config = 34; - */ - value: TransformString; - case: "transformStringConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.Passthrough passthrough_config = 35; - */ - value: Passthrough; - case: "passthroughConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.Null nullconfig = 36; - */ - value: Null; - case: "nullconfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.UserDefinedTransformerConfig user_defined_transformer_config = 37; - */ - value: UserDefinedTransformerConfig; - case: "userDefinedTransformerConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateDefault generate_default_config = 38; - */ - value: GenerateDefault; - case: "generateDefaultConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformJavascript transform_javascript_config = 39; - */ - value: TransformJavascript; - case: "transformJavascriptConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateCategorical generate_categorical_config = 40; - */ - value: GenerateCategorical; - case: "generateCategoricalConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.TransformCharacterScramble transform_character_scramble_config = 41; - */ - value: TransformCharacterScramble; - case: "transformCharacterScrambleConfig"; - } | { - /** - * @generated from field: mgmt.v1alpha1.GenerateJavascript generate_javascript_config = 42; - */ - value: GenerateJavascript; - case: "generateJavascriptConfig"; - } | { case: undefined; value?: undefined } = { case: undefined }; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformerConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "generate_email_config", kind: "message", T: GenerateEmail, oneof: "config" }, - { no: 2, name: "transform_email_config", kind: "message", T: TransformEmail, oneof: "config" }, - { no: 3, name: "generate_bool_config", kind: "message", T: GenerateBool, oneof: "config" }, - { no: 4, name: "generate_card_number_config", kind: "message", T: GenerateCardNumber, oneof: "config" }, - { no: 5, name: "generate_city_config", kind: "message", T: GenerateCity, oneof: "config" }, - { no: 6, name: "generate_e164_phone_number_config", kind: "message", T: GenerateE164PhoneNumber, oneof: "config" }, - { no: 7, name: "generate_first_name_config", kind: "message", T: GenerateFirstName, oneof: "config" }, - { no: 8, name: "generate_float64_config", kind: "message", T: GenerateFloat64, oneof: "config" }, - { no: 9, name: "generate_full_address_config", kind: "message", T: GenerateFullAddress, oneof: "config" }, - { no: 10, name: "generate_full_name_config", kind: "message", T: GenerateFullName, oneof: "config" }, - { no: 11, name: "generate_gender_config", kind: "message", T: GenerateGender, oneof: "config" }, - { no: 12, name: "generate_int64_phone_number_config", kind: "message", T: GenerateInt64PhoneNumber, oneof: "config" }, - { no: 13, name: "generate_int64_config", kind: "message", T: GenerateInt64, oneof: "config" }, - { no: 14, name: "generate_last_name_config", kind: "message", T: GenerateLastName, oneof: "config" }, - { no: 15, name: "generate_sha256hash_config", kind: "message", T: GenerateSha256Hash, oneof: "config" }, - { no: 16, name: "generate_ssn_config", kind: "message", T: GenerateSSN, oneof: "config" }, - { no: 17, name: "generate_state_config", kind: "message", T: GenerateState, oneof: "config" }, - { no: 18, name: "generate_street_address_config", kind: "message", T: GenerateStreetAddress, oneof: "config" }, - { no: 19, name: "generate_string_phone_number_config", kind: "message", T: GenerateStringPhoneNumber, oneof: "config" }, - { no: 20, name: "generate_string_config", kind: "message", T: GenerateString, oneof: "config" }, - { no: 21, name: "generate_unixtimestamp_config", kind: "message", T: GenerateUnixTimestamp, oneof: "config" }, - { no: 22, name: "generate_username_config", kind: "message", T: GenerateUsername, oneof: "config" }, - { no: 23, name: "generate_utctimestamp_config", kind: "message", T: GenerateUtcTimestamp, oneof: "config" }, - { no: 24, name: "generate_uuid_config", kind: "message", T: GenerateUuid, oneof: "config" }, - { no: 25, name: "generate_zipcode_config", kind: "message", T: GenerateZipcode, oneof: "config" }, - { no: 26, name: "transform_e164_phone_number_config", kind: "message", T: TransformE164PhoneNumber, oneof: "config" }, - { no: 27, name: "transform_first_name_config", kind: "message", T: TransformFirstName, oneof: "config" }, - { no: 28, name: "transform_float64_config", kind: "message", T: TransformFloat64, oneof: "config" }, - { no: 29, name: "transform_full_name_config", kind: "message", T: TransformFullName, oneof: "config" }, - { no: 30, name: "transform_int64_phone_number_config", kind: "message", T: TransformInt64PhoneNumber, oneof: "config" }, - { no: 31, name: "transform_int64_config", kind: "message", T: TransformInt64, oneof: "config" }, - { no: 32, name: "transform_last_name_config", kind: "message", T: TransformLastName, oneof: "config" }, - { no: 33, name: "transform_phone_number_config", kind: "message", T: TransformPhoneNumber, oneof: "config" }, - { no: 34, name: "transform_string_config", kind: "message", T: TransformString, oneof: "config" }, - { no: 35, name: "passthrough_config", kind: "message", T: Passthrough, oneof: "config" }, - { no: 36, name: "nullconfig", kind: "message", T: Null, oneof: "config" }, - { no: 37, name: "user_defined_transformer_config", kind: "message", T: UserDefinedTransformerConfig, oneof: "config" }, - { no: 38, name: "generate_default_config", kind: "message", T: GenerateDefault, oneof: "config" }, - { no: 39, name: "transform_javascript_config", kind: "message", T: TransformJavascript, oneof: "config" }, - { no: 40, name: "generate_categorical_config", kind: "message", T: GenerateCategorical, oneof: "config" }, - { no: 41, name: "transform_character_scramble_config", kind: "message", T: TransformCharacterScramble, oneof: "config" }, - { no: 42, name: "generate_javascript_config", kind: "message", T: GenerateJavascript, oneof: "config" }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformerConfig { - return new TransformerConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformerConfig { - return new TransformerConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformerConfig { - return new TransformerConfig().fromJsonString(jsonString, options); - } - - static equals(a: TransformerConfig | PlainMessage | undefined, b: TransformerConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformerConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateEmail - */ -export class GenerateEmail extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateEmail"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateEmail { - return new GenerateEmail().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateEmail { - return new GenerateEmail().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateEmail { - return new GenerateEmail().fromJsonString(jsonString, options); - } - - static equals(a: GenerateEmail | PlainMessage | undefined, b: GenerateEmail | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateEmail, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformEmail - */ -export class TransformEmail extends Message { - /** - * @generated from field: bool preserve_domain = 1; - */ - preserveDomain = false; - - /** - * @generated from field: bool preserve_length = 2; - */ - preserveLength = false; - - /** - * @generated from field: repeated string excluded_domains = 3; - */ - excludedDomains: string[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformEmail"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_domain", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "excluded_domains", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformEmail { - return new TransformEmail().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformEmail { - return new TransformEmail().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformEmail { - return new TransformEmail().fromJsonString(jsonString, options); - } - - static equals(a: TransformEmail | PlainMessage | undefined, b: TransformEmail | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformEmail, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateBool - */ -export class GenerateBool extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateBool"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateBool { - return new GenerateBool().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateBool { - return new GenerateBool().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateBool { - return new GenerateBool().fromJsonString(jsonString, options); - } - - static equals(a: GenerateBool | PlainMessage | undefined, b: GenerateBool | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateBool, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateCardNumber - */ -export class GenerateCardNumber extends Message { - /** - * @generated from field: bool valid_luhn = 1; - */ - validLuhn = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateCardNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "valid_luhn", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCardNumber { - return new GenerateCardNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCardNumber { - return new GenerateCardNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateCardNumber { - return new GenerateCardNumber().fromJsonString(jsonString, options); - } - - static equals(a: GenerateCardNumber | PlainMessage | undefined, b: GenerateCardNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateCardNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateCity - */ -export class GenerateCity extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateCity"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCity { - return new GenerateCity().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCity { - return new GenerateCity().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateCity { - return new GenerateCity().fromJsonString(jsonString, options); - } - - static equals(a: GenerateCity | PlainMessage | undefined, b: GenerateCity | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateCity, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateDefault - */ -export class GenerateDefault extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateDefault"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateDefault { - return new GenerateDefault().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateDefault { - return new GenerateDefault().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateDefault { - return new GenerateDefault().fromJsonString(jsonString, options); - } - - static equals(a: GenerateDefault | PlainMessage | undefined, b: GenerateDefault | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateDefault, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateE164PhoneNumber - */ -export class GenerateE164PhoneNumber extends Message { - /** - * @generated from field: int64 min = 1; - */ - min = protoInt64.zero; - - /** - * @generated from field: int64 max = 2; - */ - max = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateE164PhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 2, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateE164PhoneNumber { - return new GenerateE164PhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateE164PhoneNumber { - return new GenerateE164PhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateE164PhoneNumber { - return new GenerateE164PhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: GenerateE164PhoneNumber | PlainMessage | undefined, b: GenerateE164PhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateE164PhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateFirstName - */ -export class GenerateFirstName extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateFirstName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFirstName { - return new GenerateFirstName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFirstName { - return new GenerateFirstName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateFirstName { - return new GenerateFirstName().fromJsonString(jsonString, options); - } - - static equals(a: GenerateFirstName | PlainMessage | undefined, b: GenerateFirstName | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateFirstName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateFloat64 - */ -export class GenerateFloat64 extends Message { - /** - * @generated from field: bool randomize_sign = 1; - */ - randomizeSign = false; - - /** - * @generated from field: double min = 2; - */ - min = 0; - - /** - * @generated from field: double max = 3; - */ - max = 0; - - /** - * @generated from field: int64 precision = 4; - */ - precision = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateFloat64"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "randomize_sign", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "min", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, - { no: 3, name: "max", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, - { no: 4, name: "precision", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFloat64 { - return new GenerateFloat64().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFloat64 { - return new GenerateFloat64().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateFloat64 { - return new GenerateFloat64().fromJsonString(jsonString, options); - } - - static equals(a: GenerateFloat64 | PlainMessage | undefined, b: GenerateFloat64 | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateFloat64, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateFullAddress - */ -export class GenerateFullAddress extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateFullAddress"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFullAddress { - return new GenerateFullAddress().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFullAddress { - return new GenerateFullAddress().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateFullAddress { - return new GenerateFullAddress().fromJsonString(jsonString, options); - } - - static equals(a: GenerateFullAddress | PlainMessage | undefined, b: GenerateFullAddress | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateFullAddress, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateFullName - */ -export class GenerateFullName extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateFullName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFullName { - return new GenerateFullName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFullName { - return new GenerateFullName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateFullName { - return new GenerateFullName().fromJsonString(jsonString, options); - } - - static equals(a: GenerateFullName | PlainMessage | undefined, b: GenerateFullName | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateFullName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateGender - */ -export class GenerateGender extends Message { - /** - * @generated from field: bool abbreviate = 1; - */ - abbreviate = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateGender"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "abbreviate", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateGender { - return new GenerateGender().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateGender { - return new GenerateGender().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateGender { - return new GenerateGender().fromJsonString(jsonString, options); - } - - static equals(a: GenerateGender | PlainMessage | undefined, b: GenerateGender | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateGender, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateInt64PhoneNumber - */ -export class GenerateInt64PhoneNumber extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateInt64PhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateInt64PhoneNumber { - return new GenerateInt64PhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateInt64PhoneNumber { - return new GenerateInt64PhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateInt64PhoneNumber { - return new GenerateInt64PhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: GenerateInt64PhoneNumber | PlainMessage | undefined, b: GenerateInt64PhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateInt64PhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateInt64 - */ -export class GenerateInt64 extends Message { - /** - * @generated from field: bool randomize_sign = 1; - */ - randomizeSign = false; - - /** - * @generated from field: int64 min = 2; - */ - min = protoInt64.zero; - - /** - * @generated from field: int64 max = 3; - */ - max = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateInt64"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "randomize_sign", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 3, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateInt64 { - return new GenerateInt64().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateInt64 { - return new GenerateInt64().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateInt64 { - return new GenerateInt64().fromJsonString(jsonString, options); - } - - static equals(a: GenerateInt64 | PlainMessage | undefined, b: GenerateInt64 | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateInt64, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateLastName - */ -export class GenerateLastName extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateLastName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateLastName { - return new GenerateLastName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateLastName { - return new GenerateLastName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateLastName { - return new GenerateLastName().fromJsonString(jsonString, options); - } - - static equals(a: GenerateLastName | PlainMessage | undefined, b: GenerateLastName | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateLastName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateSha256Hash - */ -export class GenerateSha256Hash extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateSha256Hash"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSha256Hash { - return new GenerateSha256Hash().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSha256Hash { - return new GenerateSha256Hash().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateSha256Hash { - return new GenerateSha256Hash().fromJsonString(jsonString, options); - } - - static equals(a: GenerateSha256Hash | PlainMessage | undefined, b: GenerateSha256Hash | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateSha256Hash, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateSSN - */ -export class GenerateSSN extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateSSN"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSSN { - return new GenerateSSN().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSSN { - return new GenerateSSN().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateSSN { - return new GenerateSSN().fromJsonString(jsonString, options); - } - - static equals(a: GenerateSSN | PlainMessage | undefined, b: GenerateSSN | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateSSN, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateState - */ -export class GenerateState extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateState"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateState { - return new GenerateState().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateState { - return new GenerateState().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateState { - return new GenerateState().fromJsonString(jsonString, options); - } - - static equals(a: GenerateState | PlainMessage | undefined, b: GenerateState | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateState, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateStreetAddress - */ -export class GenerateStreetAddress extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateStreetAddress"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateStreetAddress { - return new GenerateStreetAddress().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateStreetAddress { - return new GenerateStreetAddress().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateStreetAddress { - return new GenerateStreetAddress().fromJsonString(jsonString, options); - } - - static equals(a: GenerateStreetAddress | PlainMessage | undefined, b: GenerateStreetAddress | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateStreetAddress, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateStringPhoneNumber - */ -export class GenerateStringPhoneNumber extends Message { - /** - * @generated from field: int64 min = 2; - */ - min = protoInt64.zero; - - /** - * @generated from field: int64 max = 3; - */ - max = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateStringPhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 2, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 3, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateStringPhoneNumber { - return new GenerateStringPhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateStringPhoneNumber { - return new GenerateStringPhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateStringPhoneNumber { - return new GenerateStringPhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: GenerateStringPhoneNumber | PlainMessage | undefined, b: GenerateStringPhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateStringPhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateString - */ -export class GenerateString extends Message { - /** - * @generated from field: int64 min = 1; - */ - min = protoInt64.zero; - - /** - * @generated from field: int64 max = 2; - */ - max = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateString"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 2, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateString { - return new GenerateString().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateString { - return new GenerateString().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateString { - return new GenerateString().fromJsonString(jsonString, options); - } - - static equals(a: GenerateString | PlainMessage | undefined, b: GenerateString | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateString, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateUnixTimestamp - */ -export class GenerateUnixTimestamp extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateUnixTimestamp"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUnixTimestamp { - return new GenerateUnixTimestamp().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUnixTimestamp { - return new GenerateUnixTimestamp().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateUnixTimestamp { - return new GenerateUnixTimestamp().fromJsonString(jsonString, options); - } - - static equals(a: GenerateUnixTimestamp | PlainMessage | undefined, b: GenerateUnixTimestamp | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateUnixTimestamp, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateUsername - */ -export class GenerateUsername extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateUsername"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUsername { - return new GenerateUsername().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUsername { - return new GenerateUsername().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateUsername { - return new GenerateUsername().fromJsonString(jsonString, options); - } - - static equals(a: GenerateUsername | PlainMessage | undefined, b: GenerateUsername | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateUsername, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateUtcTimestamp - */ -export class GenerateUtcTimestamp extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateUtcTimestamp"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUtcTimestamp { - return new GenerateUtcTimestamp().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUtcTimestamp { - return new GenerateUtcTimestamp().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateUtcTimestamp { - return new GenerateUtcTimestamp().fromJsonString(jsonString, options); - } - - static equals(a: GenerateUtcTimestamp | PlainMessage | undefined, b: GenerateUtcTimestamp | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateUtcTimestamp, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateUuid - */ -export class GenerateUuid extends Message { - /** - * @generated from field: bool include_hyphens = 1; - */ - includeHyphens = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateUuid"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "include_hyphens", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUuid { - return new GenerateUuid().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUuid { - return new GenerateUuid().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateUuid { - return new GenerateUuid().fromJsonString(jsonString, options); - } - - static equals(a: GenerateUuid | PlainMessage | undefined, b: GenerateUuid | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateUuid, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateZipcode - */ -export class GenerateZipcode extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateZipcode"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateZipcode { - return new GenerateZipcode().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateZipcode { - return new GenerateZipcode().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateZipcode { - return new GenerateZipcode().fromJsonString(jsonString, options); - } - - static equals(a: GenerateZipcode | PlainMessage | undefined, b: GenerateZipcode | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateZipcode, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformE164PhoneNumber - */ -export class TransformE164PhoneNumber extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformE164PhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformE164PhoneNumber { - return new TransformE164PhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformE164PhoneNumber { - return new TransformE164PhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformE164PhoneNumber { - return new TransformE164PhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: TransformE164PhoneNumber | PlainMessage | undefined, b: TransformE164PhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformE164PhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformFirstName - */ -export class TransformFirstName extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformFirstName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformFirstName { - return new TransformFirstName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformFirstName { - return new TransformFirstName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformFirstName { - return new TransformFirstName().fromJsonString(jsonString, options); - } - - static equals(a: TransformFirstName | PlainMessage | undefined, b: TransformFirstName | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformFirstName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformFloat64 - */ -export class TransformFloat64 extends Message { - /** - * @generated from field: double randomization_range_min = 1; - */ - randomizationRangeMin = 0; - - /** - * @generated from field: double randomization_range_max = 2; - */ - randomizationRangeMax = 0; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformFloat64"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "randomization_range_min", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, - { no: 2, name: "randomization_range_max", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformFloat64 { - return new TransformFloat64().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformFloat64 { - return new TransformFloat64().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformFloat64 { - return new TransformFloat64().fromJsonString(jsonString, options); - } - - static equals(a: TransformFloat64 | PlainMessage | undefined, b: TransformFloat64 | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformFloat64, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformFullName - */ -export class TransformFullName extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformFullName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformFullName { - return new TransformFullName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformFullName { - return new TransformFullName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformFullName { - return new TransformFullName().fromJsonString(jsonString, options); - } - - static equals(a: TransformFullName | PlainMessage | undefined, b: TransformFullName | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformFullName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformInt64PhoneNumber - */ -export class TransformInt64PhoneNumber extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformInt64PhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformInt64PhoneNumber { - return new TransformInt64PhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformInt64PhoneNumber { - return new TransformInt64PhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformInt64PhoneNumber { - return new TransformInt64PhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: TransformInt64PhoneNumber | PlainMessage | undefined, b: TransformInt64PhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformInt64PhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformInt64 - */ -export class TransformInt64 extends Message { - /** - * @generated from field: int64 randomization_range_min = 1; - */ - randomizationRangeMin = protoInt64.zero; - - /** - * @generated from field: int64 randomization_range_max = 2; - */ - randomizationRangeMax = protoInt64.zero; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformInt64"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "randomization_range_min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - { no: 2, name: "randomization_range_max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformInt64 { - return new TransformInt64().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformInt64 { - return new TransformInt64().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformInt64 { - return new TransformInt64().fromJsonString(jsonString, options); - } - - static equals(a: TransformInt64 | PlainMessage | undefined, b: TransformInt64 | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformInt64, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformLastName - */ -export class TransformLastName extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformLastName"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformLastName { - return new TransformLastName().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformLastName { - return new TransformLastName().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformLastName { - return new TransformLastName().fromJsonString(jsonString, options); - } - - static equals(a: TransformLastName | PlainMessage | undefined, b: TransformLastName | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformLastName, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformPhoneNumber - */ -export class TransformPhoneNumber extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformPhoneNumber"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformPhoneNumber { - return new TransformPhoneNumber().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformPhoneNumber { - return new TransformPhoneNumber().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformPhoneNumber { - return new TransformPhoneNumber().fromJsonString(jsonString, options); - } - - static equals(a: TransformPhoneNumber | PlainMessage | undefined, b: TransformPhoneNumber | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformPhoneNumber, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformString - */ -export class TransformString extends Message { - /** - * @generated from field: bool preserve_length = 1; - */ - preserveLength = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformString"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformString { - return new TransformString().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformString { - return new TransformString().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformString { - return new TransformString().fromJsonString(jsonString, options); - } - - static equals(a: TransformString | PlainMessage | undefined, b: TransformString | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformString, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.Passthrough - */ -export class Passthrough extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.Passthrough"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): Passthrough { - return new Passthrough().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): Passthrough { - return new Passthrough().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): Passthrough { - return new Passthrough().fromJsonString(jsonString, options); - } - - static equals(a: Passthrough | PlainMessage | undefined, b: Passthrough | PlainMessage | undefined): boolean { - return proto3.util.equals(Passthrough, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.Null - */ -export class Null extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.Null"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): Null { - return new Null().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): Null { - return new Null().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): Null { - return new Null().fromJsonString(jsonString, options); - } - - static equals(a: Null | PlainMessage | undefined, b: Null | PlainMessage | undefined): boolean { - return proto3.util.equals(Null, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformJavascript - */ -export class TransformJavascript extends Message { - /** - * @generated from field: string code = 1; - */ - code = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformJavascript"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformJavascript { - return new TransformJavascript().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformJavascript { - return new TransformJavascript().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformJavascript { - return new TransformJavascript().fromJsonString(jsonString, options); - } - - static equals(a: TransformJavascript | PlainMessage | undefined, b: TransformJavascript | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformJavascript, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UserDefinedTransformerConfig - */ -export class UserDefinedTransformerConfig extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UserDefinedTransformerConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UserDefinedTransformerConfig { - return new UserDefinedTransformerConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UserDefinedTransformerConfig { - return new UserDefinedTransformerConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UserDefinedTransformerConfig { - return new UserDefinedTransformerConfig().fromJsonString(jsonString, options); - } - - static equals(a: UserDefinedTransformerConfig | PlainMessage | undefined, b: UserDefinedTransformerConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(UserDefinedTransformerConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ValidateUserJavascriptCodeRequest - */ -export class ValidateUserJavascriptCodeRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string code = 2; - */ - code = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserJavascriptCodeRequest { - return new ValidateUserJavascriptCodeRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserJavascriptCodeRequest { - return new ValidateUserJavascriptCodeRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ValidateUserJavascriptCodeRequest { - return new ValidateUserJavascriptCodeRequest().fromJsonString(jsonString, options); - } - - static equals(a: ValidateUserJavascriptCodeRequest | PlainMessage | undefined, b: ValidateUserJavascriptCodeRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(ValidateUserJavascriptCodeRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ValidateUserJavascriptCodeResponse - */ -export class ValidateUserJavascriptCodeResponse extends Message { - /** - * @generated from field: bool valid = 1; - */ - valid = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserJavascriptCodeResponse { - return new ValidateUserJavascriptCodeResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserJavascriptCodeResponse { - return new ValidateUserJavascriptCodeResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ValidateUserJavascriptCodeResponse { - return new ValidateUserJavascriptCodeResponse().fromJsonString(jsonString, options); - } - - static equals(a: ValidateUserJavascriptCodeResponse | PlainMessage | undefined, b: ValidateUserJavascriptCodeResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(ValidateUserJavascriptCodeResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateCategorical - */ -export class GenerateCategorical extends Message { - /** - * @generated from field: string categories = 1; - */ - categories = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateCategorical"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "categories", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCategorical { - return new GenerateCategorical().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCategorical { - return new GenerateCategorical().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateCategorical { - return new GenerateCategorical().fromJsonString(jsonString, options); - } - - static equals(a: GenerateCategorical | PlainMessage | undefined, b: GenerateCategorical | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateCategorical, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.TransformCharacterScramble - */ -export class TransformCharacterScramble extends Message { - /** - * a user provided regular expression that they wish to validate if it compiles in GO - * - * @generated from field: optional string user_provided_regex = 1; - */ - userProvidedRegex?: string; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.TransformCharacterScramble"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "user_provided_regex", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): TransformCharacterScramble { - return new TransformCharacterScramble().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): TransformCharacterScramble { - return new TransformCharacterScramble().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): TransformCharacterScramble { - return new TransformCharacterScramble().fromJsonString(jsonString, options); - } - - static equals(a: TransformCharacterScramble | PlainMessage | undefined, b: TransformCharacterScramble | PlainMessage | undefined): boolean { - return proto3.util.equals(TransformCharacterScramble, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GenerateJavascript - */ -export class GenerateJavascript extends Message { - /** - * @generated from field: string code = 1; - */ - code = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GenerateJavascript"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GenerateJavascript { - return new GenerateJavascript().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GenerateJavascript { - return new GenerateJavascript().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GenerateJavascript { - return new GenerateJavascript().fromJsonString(jsonString, options); - } - - static equals(a: GenerateJavascript | PlainMessage | undefined, b: GenerateJavascript | PlainMessage | undefined): boolean { - return proto3.util.equals(GenerateJavascript, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ValidateUserRegexCodeRequest - */ -export class ValidateUserRegexCodeRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string user_provided_regex = 2; - */ - userProvidedRegex = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ValidateUserRegexCodeRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "user_provided_regex", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserRegexCodeRequest { - return new ValidateUserRegexCodeRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserRegexCodeRequest { - return new ValidateUserRegexCodeRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ValidateUserRegexCodeRequest { - return new ValidateUserRegexCodeRequest().fromJsonString(jsonString, options); - } - - static equals(a: ValidateUserRegexCodeRequest | PlainMessage | undefined, b: ValidateUserRegexCodeRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(ValidateUserRegexCodeRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ValidateUserRegexCodeResponse - */ -export class ValidateUserRegexCodeResponse extends Message { - /** - * @generated from field: bool valid = 1; - */ - valid = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ValidateUserRegexCodeResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserRegexCodeResponse { - return new ValidateUserRegexCodeResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserRegexCodeResponse { - return new ValidateUserRegexCodeResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ValidateUserRegexCodeResponse { - return new ValidateUserRegexCodeResponse().fromJsonString(jsonString, options); - } - - static equals(a: ValidateUserRegexCodeResponse | PlainMessage | undefined, b: ValidateUserRegexCodeResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(ValidateUserRegexCodeResponse, a, b); - } -} - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts deleted file mode 100644 index 9183e759ab..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts +++ /dev/null @@ -1,179 +0,0 @@ -// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/user_account.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import { AcceptTeamAccountInviteRequest, AcceptTeamAccountInviteResponse, ConvertPersonalToTeamAccountRequest, ConvertPersonalToTeamAccountResponse, CreateTeamAccountRequest, CreateTeamAccountResponse, GetAccountOnboardingConfigRequest, GetAccountOnboardingConfigResponse, GetAccountTemporalConfigRequest, GetAccountTemporalConfigResponse, GetSystemInformationRequest, GetSystemInformationResponse, GetTeamAccountInvitesRequest, GetTeamAccountInvitesResponse, GetTeamAccountMembersRequest, GetTeamAccountMembersResponse, GetUserAccountsRequest, GetUserAccountsResponse, GetUserRequest, GetUserResponse, InviteUserToTeamAccountRequest, InviteUserToTeamAccountResponse, IsUserInAccountRequest, IsUserInAccountResponse, RemoveTeamAccountInviteRequest, RemoveTeamAccountInviteResponse, RemoveTeamAccountMemberRequest, RemoveTeamAccountMemberResponse, SetAccountOnboardingConfigRequest, SetAccountOnboardingConfigResponse, SetAccountTemporalConfigRequest, SetAccountTemporalConfigResponse, SetPersonalAccountRequest, SetPersonalAccountResponse, SetUserRequest, SetUserResponse } from "./user_account_pb.js"; -import { MethodKind } from "@bufbuild/protobuf"; - -/** - * @generated from service mgmt.v1alpha1.UserAccountService - */ -export const UserAccountService = { - typeName: "mgmt.v1alpha1.UserAccountService", - methods: { - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetUser - */ - getUser: { - name: "GetUser", - I: GetUserRequest, - O: GetUserResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.SetUser - */ - setUser: { - name: "SetUser", - I: SetUserRequest, - O: SetUserResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetUserAccounts - */ - getUserAccounts: { - name: "GetUserAccounts", - I: GetUserAccountsRequest, - O: GetUserAccountsResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.SetPersonalAccount - */ - setPersonalAccount: { - name: "SetPersonalAccount", - I: SetPersonalAccountRequest, - O: SetPersonalAccountResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.ConvertPersonalToTeamAccount - */ - convertPersonalToTeamAccount: { - name: "ConvertPersonalToTeamAccount", - I: ConvertPersonalToTeamAccountRequest, - O: ConvertPersonalToTeamAccountResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.CreateTeamAccount - */ - createTeamAccount: { - name: "CreateTeamAccount", - I: CreateTeamAccountRequest, - O: CreateTeamAccountResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.IsUserInAccount - */ - isUserInAccount: { - name: "IsUserInAccount", - I: IsUserInAccountRequest, - O: IsUserInAccountResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetAccountTemporalConfig - */ - getAccountTemporalConfig: { - name: "GetAccountTemporalConfig", - I: GetAccountTemporalConfigRequest, - O: GetAccountTemporalConfigResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.SetAccountTemporalConfig - */ - setAccountTemporalConfig: { - name: "SetAccountTemporalConfig", - I: SetAccountTemporalConfigRequest, - O: SetAccountTemporalConfigResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetTeamAccountMembers - */ - getTeamAccountMembers: { - name: "GetTeamAccountMembers", - I: GetTeamAccountMembersRequest, - O: GetTeamAccountMembersResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.RemoveTeamAccountMember - */ - removeTeamAccountMember: { - name: "RemoveTeamAccountMember", - I: RemoveTeamAccountMemberRequest, - O: RemoveTeamAccountMemberResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.InviteUserToTeamAccount - */ - inviteUserToTeamAccount: { - name: "InviteUserToTeamAccount", - I: InviteUserToTeamAccountRequest, - O: InviteUserToTeamAccountResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetTeamAccountInvites - */ - getTeamAccountInvites: { - name: "GetTeamAccountInvites", - I: GetTeamAccountInvitesRequest, - O: GetTeamAccountInvitesResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.RemoveTeamAccountInvite - */ - removeTeamAccountInvite: { - name: "RemoveTeamAccountInvite", - I: RemoveTeamAccountInviteRequest, - O: RemoveTeamAccountInviteResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.AcceptTeamAccountInvite - */ - acceptTeamAccountInvite: { - name: "AcceptTeamAccountInvite", - I: AcceptTeamAccountInviteRequest, - O: AcceptTeamAccountInviteResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetSystemInformation - */ - getSystemInformation: { - name: "GetSystemInformation", - I: GetSystemInformationRequest, - O: GetSystemInformationResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.GetAccountOnboardingConfig - */ - getAccountOnboardingConfig: { - name: "GetAccountOnboardingConfig", - I: GetAccountOnboardingConfigRequest, - O: GetAccountOnboardingConfigResponse, - kind: MethodKind.Unary, - }, - /** - * @generated from rpc mgmt.v1alpha1.UserAccountService.SetAccountOnboardingConfig - */ - setAccountOnboardingConfig: { - name: "SetAccountOnboardingConfig", - I: SetAccountOnboardingConfigRequest, - O: SetAccountOnboardingConfigResponse, - kind: MethodKind.Unary, - }, - } -} as const; - diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts deleted file mode 100644 index 2a384a897e..0000000000 --- a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts +++ /dev/null @@ -1,1659 +0,0 @@ -// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" -// @generated from file mgmt/v1alpha1/user_account.proto (package mgmt.v1alpha1, syntax proto3) -/* eslint-disable */ -// @ts-nocheck - -import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; -import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; - -/** - * @generated from enum mgmt.v1alpha1.UserAccountType - */ -export enum UserAccountType { - /** - * @generated from enum value: USER_ACCOUNT_TYPE_UNSPECIFIED = 0; - */ - UNSPECIFIED = 0, - - /** - * @generated from enum value: USER_ACCOUNT_TYPE_PERSONAL = 1; - */ - PERSONAL = 1, - - /** - * @generated from enum value: USER_ACCOUNT_TYPE_TEAM = 2; - */ - TEAM = 2, - - /** - * @generated from enum value: USER_ACCOUNT_TYPE_ENTERPRISE = 3; - */ - ENTERPRISE = 3, -} -// Retrieve enum metadata with: proto3.getEnumType(UserAccountType) -proto3.util.setEnumType(UserAccountType, "mgmt.v1alpha1.UserAccountType", [ - { no: 0, name: "USER_ACCOUNT_TYPE_UNSPECIFIED" }, - { no: 1, name: "USER_ACCOUNT_TYPE_PERSONAL" }, - { no: 2, name: "USER_ACCOUNT_TYPE_TEAM" }, - { no: 3, name: "USER_ACCOUNT_TYPE_ENTERPRISE" }, -]); - -/** - * @generated from message mgmt.v1alpha1.GetUserRequest - */ -export class GetUserRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserRequest { - return new GetUserRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserRequest { - return new GetUserRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserRequest { - return new GetUserRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetUserRequest | PlainMessage | undefined, b: GetUserRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserResponse - */ -export class GetUserResponse extends Message { - /** - * @generated from field: string user_id = 1; - */ - userId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserResponse { - return new GetUserResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserResponse { - return new GetUserResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserResponse { - return new GetUserResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetUserResponse | PlainMessage | undefined, b: GetUserResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetUserRequest - */ -export class SetUserRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetUserRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetUserRequest { - return new SetUserRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetUserRequest { - return new SetUserRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetUserRequest { - return new SetUserRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetUserRequest | PlainMessage | undefined, b: SetUserRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetUserRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetUserResponse - */ -export class SetUserResponse extends Message { - /** - * @generated from field: string user_id = 1; - */ - userId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetUserResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetUserResponse { - return new SetUserResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetUserResponse { - return new SetUserResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetUserResponse { - return new SetUserResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetUserResponse | PlainMessage | undefined, b: SetUserResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetUserResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserAccountsRequest - */ -export class GetUserAccountsRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserAccountsRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserAccountsRequest { - return new GetUserAccountsRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserAccountsRequest { - return new GetUserAccountsRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserAccountsRequest { - return new GetUserAccountsRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetUserAccountsRequest | PlainMessage | undefined, b: GetUserAccountsRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserAccountsRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetUserAccountsResponse - */ -export class GetUserAccountsResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.UserAccount accounts = 1; - */ - accounts: UserAccount[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetUserAccountsResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "accounts", kind: "message", T: UserAccount, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetUserAccountsResponse { - return new GetUserAccountsResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetUserAccountsResponse { - return new GetUserAccountsResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetUserAccountsResponse { - return new GetUserAccountsResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetUserAccountsResponse | PlainMessage | undefined, b: GetUserAccountsResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetUserAccountsResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.UserAccount - */ -export class UserAccount extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: mgmt.v1alpha1.UserAccountType type = 3; - */ - type = UserAccountType.UNSPECIFIED; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.UserAccount"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "type", kind: "enum", T: proto3.getEnumType(UserAccountType) }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): UserAccount { - return new UserAccount().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): UserAccount { - return new UserAccount().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): UserAccount { - return new UserAccount().fromJsonString(jsonString, options); - } - - static equals(a: UserAccount | PlainMessage | undefined, b: UserAccount | PlainMessage | undefined): boolean { - return proto3.util.equals(UserAccount, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest - */ -export class ConvertPersonalToTeamAccountRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConvertPersonalToTeamAccountRequest { - return new ConvertPersonalToTeamAccountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConvertPersonalToTeamAccountRequest { - return new ConvertPersonalToTeamAccountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConvertPersonalToTeamAccountRequest { - return new ConvertPersonalToTeamAccountRequest().fromJsonString(jsonString, options); - } - - static equals(a: ConvertPersonalToTeamAccountRequest | PlainMessage | undefined, b: ConvertPersonalToTeamAccountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(ConvertPersonalToTeamAccountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse - */ -export class ConvertPersonalToTeamAccountResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): ConvertPersonalToTeamAccountResponse { - return new ConvertPersonalToTeamAccountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): ConvertPersonalToTeamAccountResponse { - return new ConvertPersonalToTeamAccountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): ConvertPersonalToTeamAccountResponse { - return new ConvertPersonalToTeamAccountResponse().fromJsonString(jsonString, options); - } - - static equals(a: ConvertPersonalToTeamAccountResponse | PlainMessage | undefined, b: ConvertPersonalToTeamAccountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(ConvertPersonalToTeamAccountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetPersonalAccountRequest - */ -export class SetPersonalAccountRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetPersonalAccountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetPersonalAccountRequest { - return new SetPersonalAccountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetPersonalAccountRequest { - return new SetPersonalAccountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetPersonalAccountRequest { - return new SetPersonalAccountRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetPersonalAccountRequest | PlainMessage | undefined, b: SetPersonalAccountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetPersonalAccountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetPersonalAccountResponse - */ -export class SetPersonalAccountResponse extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetPersonalAccountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetPersonalAccountResponse { - return new SetPersonalAccountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetPersonalAccountResponse { - return new SetPersonalAccountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetPersonalAccountResponse { - return new SetPersonalAccountResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetPersonalAccountResponse | PlainMessage | undefined, b: SetPersonalAccountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetPersonalAccountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsUserInAccountRequest - */ -export class IsUserInAccountRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsUserInAccountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsUserInAccountRequest { - return new IsUserInAccountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsUserInAccountRequest { - return new IsUserInAccountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsUserInAccountRequest { - return new IsUserInAccountRequest().fromJsonString(jsonString, options); - } - - static equals(a: IsUserInAccountRequest | PlainMessage | undefined, b: IsUserInAccountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(IsUserInAccountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.IsUserInAccountResponse - */ -export class IsUserInAccountResponse extends Message { - /** - * @generated from field: bool ok = 1; - */ - ok = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.IsUserInAccountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "ok", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): IsUserInAccountResponse { - return new IsUserInAccountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): IsUserInAccountResponse { - return new IsUserInAccountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): IsUserInAccountResponse { - return new IsUserInAccountResponse().fromJsonString(jsonString, options); - } - - static equals(a: IsUserInAccountResponse | PlainMessage | undefined, b: IsUserInAccountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(IsUserInAccountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountTemporalConfigRequest - */ -export class GetAccountTemporalConfigRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountTemporalConfigRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountTemporalConfigRequest { - return new GetAccountTemporalConfigRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountTemporalConfigRequest { - return new GetAccountTemporalConfigRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountTemporalConfigRequest { - return new GetAccountTemporalConfigRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountTemporalConfigRequest | PlainMessage | undefined, b: GetAccountTemporalConfigRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountTemporalConfigRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountTemporalConfigResponse - */ -export class GetAccountTemporalConfigResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 1; - */ - config?: AccountTemporalConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountTemporalConfigResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "config", kind: "message", T: AccountTemporalConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountTemporalConfigResponse { - return new GetAccountTemporalConfigResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountTemporalConfigResponse { - return new GetAccountTemporalConfigResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountTemporalConfigResponse { - return new GetAccountTemporalConfigResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountTemporalConfigResponse | PlainMessage | undefined, b: GetAccountTemporalConfigResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountTemporalConfigResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetAccountTemporalConfigRequest - */ -export class SetAccountTemporalConfigRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 2; - */ - config?: AccountTemporalConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetAccountTemporalConfigRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "config", kind: "message", T: AccountTemporalConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountTemporalConfigRequest { - return new SetAccountTemporalConfigRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountTemporalConfigRequest { - return new SetAccountTemporalConfigRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetAccountTemporalConfigRequest { - return new SetAccountTemporalConfigRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetAccountTemporalConfigRequest | PlainMessage | undefined, b: SetAccountTemporalConfigRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetAccountTemporalConfigRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetAccountTemporalConfigResponse - */ -export class SetAccountTemporalConfigResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 1; - */ - config?: AccountTemporalConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetAccountTemporalConfigResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "config", kind: "message", T: AccountTemporalConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountTemporalConfigResponse { - return new SetAccountTemporalConfigResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountTemporalConfigResponse { - return new SetAccountTemporalConfigResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetAccountTemporalConfigResponse { - return new SetAccountTemporalConfigResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetAccountTemporalConfigResponse | PlainMessage | undefined, b: SetAccountTemporalConfigResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetAccountTemporalConfigResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AccountTemporalConfig - */ -export class AccountTemporalConfig extends Message { - /** - * @generated from field: string url = 1; - */ - url = ""; - - /** - * @generated from field: string namespace = 2; - */ - namespace = ""; - - /** - * @generated from field: string sync_job_queue_name = 3; - */ - syncJobQueueName = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccountTemporalConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "namespace", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "sync_job_queue_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccountTemporalConfig { - return new AccountTemporalConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccountTemporalConfig { - return new AccountTemporalConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccountTemporalConfig { - return new AccountTemporalConfig().fromJsonString(jsonString, options); - } - - static equals(a: AccountTemporalConfig | PlainMessage | undefined, b: AccountTemporalConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(AccountTemporalConfig, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateTeamAccountRequest - */ -export class CreateTeamAccountRequest extends Message { - /** - * @generated from field: string name = 1; - */ - name = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateTeamAccountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateTeamAccountRequest { - return new CreateTeamAccountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateTeamAccountRequest { - return new CreateTeamAccountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateTeamAccountRequest { - return new CreateTeamAccountRequest().fromJsonString(jsonString, options); - } - - static equals(a: CreateTeamAccountRequest | PlainMessage | undefined, b: CreateTeamAccountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateTeamAccountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.CreateTeamAccountResponse - */ -export class CreateTeamAccountResponse extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.CreateTeamAccountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): CreateTeamAccountResponse { - return new CreateTeamAccountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): CreateTeamAccountResponse { - return new CreateTeamAccountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): CreateTeamAccountResponse { - return new CreateTeamAccountResponse().fromJsonString(jsonString, options); - } - - static equals(a: CreateTeamAccountResponse | PlainMessage | undefined, b: CreateTeamAccountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(CreateTeamAccountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AccountUser - */ -export class AccountUser extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string name = 2; - */ - name = ""; - - /** - * @generated from field: string image = 3; - */ - image = ""; - - /** - * @generated from field: string email = 4; - */ - email = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccountUser"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "image", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccountUser { - return new AccountUser().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccountUser { - return new AccountUser().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccountUser { - return new AccountUser().fromJsonString(jsonString, options); - } - - static equals(a: AccountUser | PlainMessage | undefined, b: AccountUser | PlainMessage | undefined): boolean { - return proto3.util.equals(AccountUser, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetTeamAccountMembersRequest - */ -export class GetTeamAccountMembersRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetTeamAccountMembersRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountMembersRequest { - return new GetTeamAccountMembersRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountMembersRequest { - return new GetTeamAccountMembersRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountMembersRequest { - return new GetTeamAccountMembersRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetTeamAccountMembersRequest | PlainMessage | undefined, b: GetTeamAccountMembersRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetTeamAccountMembersRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetTeamAccountMembersResponse - */ -export class GetTeamAccountMembersResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.AccountUser users = 1; - */ - users: AccountUser[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetTeamAccountMembersResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "users", kind: "message", T: AccountUser, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountMembersResponse { - return new GetTeamAccountMembersResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountMembersResponse { - return new GetTeamAccountMembersResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountMembersResponse { - return new GetTeamAccountMembersResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetTeamAccountMembersResponse | PlainMessage | undefined, b: GetTeamAccountMembersResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetTeamAccountMembersResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RemoveTeamAccountMemberRequest - */ -export class RemoveTeamAccountMemberRequest extends Message { - /** - * @generated from field: string user_id = 1; - */ - userId = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountMemberRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountMemberRequest { - return new RemoveTeamAccountMemberRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountMemberRequest { - return new RemoveTeamAccountMemberRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountMemberRequest { - return new RemoveTeamAccountMemberRequest().fromJsonString(jsonString, options); - } - - static equals(a: RemoveTeamAccountMemberRequest | PlainMessage | undefined, b: RemoveTeamAccountMemberRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(RemoveTeamAccountMemberRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RemoveTeamAccountMemberResponse - */ -export class RemoveTeamAccountMemberResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountMemberResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountMemberResponse { - return new RemoveTeamAccountMemberResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountMemberResponse { - return new RemoveTeamAccountMemberResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountMemberResponse { - return new RemoveTeamAccountMemberResponse().fromJsonString(jsonString, options); - } - - static equals(a: RemoveTeamAccountMemberResponse | PlainMessage | undefined, b: RemoveTeamAccountMemberResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(RemoveTeamAccountMemberResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.InviteUserToTeamAccountRequest - */ -export class InviteUserToTeamAccountRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: string email = 2; - */ - email = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.InviteUserToTeamAccountRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): InviteUserToTeamAccountRequest { - return new InviteUserToTeamAccountRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): InviteUserToTeamAccountRequest { - return new InviteUserToTeamAccountRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): InviteUserToTeamAccountRequest { - return new InviteUserToTeamAccountRequest().fromJsonString(jsonString, options); - } - - static equals(a: InviteUserToTeamAccountRequest | PlainMessage | undefined, b: InviteUserToTeamAccountRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(InviteUserToTeamAccountRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AccountInvite - */ -export class AccountInvite extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - /** - * @generated from field: string account_id = 2; - */ - accountId = ""; - - /** - * @generated from field: string sender_user_id = 3; - */ - senderUserId = ""; - - /** - * @generated from field: string email = 4; - */ - email = ""; - - /** - * @generated from field: string token = 5; - */ - token = ""; - - /** - * @generated from field: bool accepted = 6; - */ - accepted = false; - - /** - * @generated from field: google.protobuf.Timestamp created_at = 7; - */ - createdAt?: Timestamp; - - /** - * @generated from field: google.protobuf.Timestamp updated_at = 8; - */ - updatedAt?: Timestamp; - - /** - * @generated from field: google.protobuf.Timestamp expires_at = 9; - */ - expiresAt?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccountInvite"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "sender_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 6, name: "accepted", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 7, name: "created_at", kind: "message", T: Timestamp }, - { no: 8, name: "updated_at", kind: "message", T: Timestamp }, - { no: 9, name: "expires_at", kind: "message", T: Timestamp }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccountInvite { - return new AccountInvite().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccountInvite { - return new AccountInvite().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccountInvite { - return new AccountInvite().fromJsonString(jsonString, options); - } - - static equals(a: AccountInvite | PlainMessage | undefined, b: AccountInvite | PlainMessage | undefined): boolean { - return proto3.util.equals(AccountInvite, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.InviteUserToTeamAccountResponse - */ -export class InviteUserToTeamAccountResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountInvite invite = 1; - */ - invite?: AccountInvite; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.InviteUserToTeamAccountResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "invite", kind: "message", T: AccountInvite }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): InviteUserToTeamAccountResponse { - return new InviteUserToTeamAccountResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): InviteUserToTeamAccountResponse { - return new InviteUserToTeamAccountResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): InviteUserToTeamAccountResponse { - return new InviteUserToTeamAccountResponse().fromJsonString(jsonString, options); - } - - static equals(a: InviteUserToTeamAccountResponse | PlainMessage | undefined, b: InviteUserToTeamAccountResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(InviteUserToTeamAccountResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetTeamAccountInvitesRequest - */ -export class GetTeamAccountInvitesRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetTeamAccountInvitesRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountInvitesRequest { - return new GetTeamAccountInvitesRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountInvitesRequest { - return new GetTeamAccountInvitesRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountInvitesRequest { - return new GetTeamAccountInvitesRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetTeamAccountInvitesRequest | PlainMessage | undefined, b: GetTeamAccountInvitesRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetTeamAccountInvitesRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetTeamAccountInvitesResponse - */ -export class GetTeamAccountInvitesResponse extends Message { - /** - * @generated from field: repeated mgmt.v1alpha1.AccountInvite invites = 1; - */ - invites: AccountInvite[] = []; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetTeamAccountInvitesResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "invites", kind: "message", T: AccountInvite, repeated: true }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountInvitesResponse { - return new GetTeamAccountInvitesResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountInvitesResponse { - return new GetTeamAccountInvitesResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountInvitesResponse { - return new GetTeamAccountInvitesResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetTeamAccountInvitesResponse | PlainMessage | undefined, b: GetTeamAccountInvitesResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetTeamAccountInvitesResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RemoveTeamAccountInviteRequest - */ -export class RemoveTeamAccountInviteRequest extends Message { - /** - * @generated from field: string id = 1; - */ - id = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountInviteRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountInviteRequest { - return new RemoveTeamAccountInviteRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountInviteRequest { - return new RemoveTeamAccountInviteRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountInviteRequest { - return new RemoveTeamAccountInviteRequest().fromJsonString(jsonString, options); - } - - static equals(a: RemoveTeamAccountInviteRequest | PlainMessage | undefined, b: RemoveTeamAccountInviteRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(RemoveTeamAccountInviteRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.RemoveTeamAccountInviteResponse - */ -export class RemoveTeamAccountInviteResponse extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountInviteResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountInviteResponse { - return new RemoveTeamAccountInviteResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountInviteResponse { - return new RemoveTeamAccountInviteResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountInviteResponse { - return new RemoveTeamAccountInviteResponse().fromJsonString(jsonString, options); - } - - static equals(a: RemoveTeamAccountInviteResponse | PlainMessage | undefined, b: RemoveTeamAccountInviteResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(RemoveTeamAccountInviteResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AcceptTeamAccountInviteRequest - */ -export class AcceptTeamAccountInviteRequest extends Message { - /** - * @generated from field: string token = 1; - */ - token = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AcceptTeamAccountInviteRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AcceptTeamAccountInviteRequest { - return new AcceptTeamAccountInviteRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AcceptTeamAccountInviteRequest { - return new AcceptTeamAccountInviteRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AcceptTeamAccountInviteRequest { - return new AcceptTeamAccountInviteRequest().fromJsonString(jsonString, options); - } - - static equals(a: AcceptTeamAccountInviteRequest | PlainMessage | undefined, b: AcceptTeamAccountInviteRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(AcceptTeamAccountInviteRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AcceptTeamAccountInviteResponse - */ -export class AcceptTeamAccountInviteResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.UserAccount account = 1; - */ - account?: UserAccount; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AcceptTeamAccountInviteResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account", kind: "message", T: UserAccount }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AcceptTeamAccountInviteResponse { - return new AcceptTeamAccountInviteResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AcceptTeamAccountInviteResponse { - return new AcceptTeamAccountInviteResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AcceptTeamAccountInviteResponse { - return new AcceptTeamAccountInviteResponse().fromJsonString(jsonString, options); - } - - static equals(a: AcceptTeamAccountInviteResponse | PlainMessage | undefined, b: AcceptTeamAccountInviteResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(AcceptTeamAccountInviteResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetSystemInformationRequest - */ -export class GetSystemInformationRequest extends Message { - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemInformationRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemInformationRequest { - return new GetSystemInformationRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemInformationRequest { - return new GetSystemInformationRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemInformationRequest { - return new GetSystemInformationRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemInformationRequest | PlainMessage | undefined, b: GetSystemInformationRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemInformationRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetSystemInformationResponse - */ -export class GetSystemInformationResponse extends Message { - /** - * @generated from field: string version = 1; - */ - version = ""; - - /** - * @generated from field: string commit = 2; - */ - commit = ""; - - /** - * @generated from field: string compiler = 3; - */ - compiler = ""; - - /** - * @generated from field: string platform = 4; - */ - platform = ""; - - /** - * @generated from field: google.protobuf.Timestamp build_date = 5; - */ - buildDate?: Timestamp; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetSystemInformationResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "version", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "commit", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 3, name: "compiler", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 4, name: "platform", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 5, name: "build_date", kind: "message", T: Timestamp }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemInformationResponse { - return new GetSystemInformationResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemInformationResponse { - return new GetSystemInformationResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetSystemInformationResponse { - return new GetSystemInformationResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetSystemInformationResponse | PlainMessage | undefined, b: GetSystemInformationResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetSystemInformationResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountOnboardingConfigRequest - */ -export class GetAccountOnboardingConfigRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountOnboardingConfigRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountOnboardingConfigRequest { - return new GetAccountOnboardingConfigRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountOnboardingConfigRequest { - return new GetAccountOnboardingConfigRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountOnboardingConfigRequest { - return new GetAccountOnboardingConfigRequest().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountOnboardingConfigRequest | PlainMessage | undefined, b: GetAccountOnboardingConfigRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountOnboardingConfigRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.GetAccountOnboardingConfigResponse - */ -export class GetAccountOnboardingConfigResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 1; - */ - config?: AccountOnboardingConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.GetAccountOnboardingConfigResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "config", kind: "message", T: AccountOnboardingConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountOnboardingConfigResponse { - return new GetAccountOnboardingConfigResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountOnboardingConfigResponse { - return new GetAccountOnboardingConfigResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): GetAccountOnboardingConfigResponse { - return new GetAccountOnboardingConfigResponse().fromJsonString(jsonString, options); - } - - static equals(a: GetAccountOnboardingConfigResponse | PlainMessage | undefined, b: GetAccountOnboardingConfigResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(GetAccountOnboardingConfigResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetAccountOnboardingConfigRequest - */ -export class SetAccountOnboardingConfigRequest extends Message { - /** - * @generated from field: string account_id = 1; - */ - accountId = ""; - - /** - * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 2; - */ - config?: AccountOnboardingConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetAccountOnboardingConfigRequest"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, - { no: 2, name: "config", kind: "message", T: AccountOnboardingConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountOnboardingConfigRequest { - return new SetAccountOnboardingConfigRequest().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountOnboardingConfigRequest { - return new SetAccountOnboardingConfigRequest().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetAccountOnboardingConfigRequest { - return new SetAccountOnboardingConfigRequest().fromJsonString(jsonString, options); - } - - static equals(a: SetAccountOnboardingConfigRequest | PlainMessage | undefined, b: SetAccountOnboardingConfigRequest | PlainMessage | undefined): boolean { - return proto3.util.equals(SetAccountOnboardingConfigRequest, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.SetAccountOnboardingConfigResponse - */ -export class SetAccountOnboardingConfigResponse extends Message { - /** - * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 1; - */ - config?: AccountOnboardingConfig; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.SetAccountOnboardingConfigResponse"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "config", kind: "message", T: AccountOnboardingConfig }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountOnboardingConfigResponse { - return new SetAccountOnboardingConfigResponse().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountOnboardingConfigResponse { - return new SetAccountOnboardingConfigResponse().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): SetAccountOnboardingConfigResponse { - return new SetAccountOnboardingConfigResponse().fromJsonString(jsonString, options); - } - - static equals(a: SetAccountOnboardingConfigResponse | PlainMessage | undefined, b: SetAccountOnboardingConfigResponse | PlainMessage | undefined): boolean { - return proto3.util.equals(SetAccountOnboardingConfigResponse, a, b); - } -} - -/** - * @generated from message mgmt.v1alpha1.AccountOnboardingConfig - */ -export class AccountOnboardingConfig extends Message { - /** - * @generated from field: bool has_created_source_connection = 1; - */ - hasCreatedSourceConnection = false; - - /** - * @generated from field: bool has_created_destination_connection = 2; - */ - hasCreatedDestinationConnection = false; - - /** - * @generated from field: bool has_created_job = 3; - */ - hasCreatedJob = false; - - /** - * @generated from field: bool has_invited_members = 4; - */ - hasInvitedMembers = false; - - constructor(data?: PartialMessage) { - super(); - proto3.util.initPartial(data, this); - } - - static readonly runtime: typeof proto3 = proto3; - static readonly typeName = "mgmt.v1alpha1.AccountOnboardingConfig"; - static readonly fields: FieldList = proto3.util.newFieldList(() => [ - { no: 1, name: "has_created_source_connection", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 2, name: "has_created_destination_connection", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 3, name: "has_created_job", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - { no: 4, name: "has_invited_members", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, - ]); - - static fromBinary(bytes: Uint8Array, options?: Partial): AccountOnboardingConfig { - return new AccountOnboardingConfig().fromBinary(bytes, options); - } - - static fromJson(jsonValue: JsonValue, options?: Partial): AccountOnboardingConfig { - return new AccountOnboardingConfig().fromJson(jsonValue, options); - } - - static fromJsonString(jsonString: string, options?: Partial): AccountOnboardingConfig { - return new AccountOnboardingConfig().fromJsonString(jsonString, options); - } - - static equals(a: AccountOnboardingConfig | PlainMessage | undefined, b: AccountOnboardingConfig | PlainMessage | undefined): boolean { - return proto3.util.equals(AccountOnboardingConfig, a, b); - } -} - diff --git a/worker/internal/benthos/config.go b/worker/internal/benthos/config.go index e645f5efa2..e603fec7ba 100644 --- a/worker/internal/benthos/config.go +++ b/worker/internal/benthos/config.go @@ -196,8 +196,9 @@ type SwitchOutputCase struct { Output Outputs `json:"output,omitempty" yaml:"output,omitempty"` } type ErrorOutputConfig struct { - ErrorMsg string `json:"error_msg" yaml:"error_msg"` - Batching *Batching `json:"batching,omitempty" yaml:"batching,omitempty"` + ErrorMsg string `json:"error_msg" yaml:"error_msg"` + MaxRetries *int `json:"max_retries,omitempty" yaml:"max_retries,omitempty"` + Batching *Batching `json:"batching,omitempty" yaml:"batching,omitempty"` } type RedisHashOutputConfig struct { diff --git a/worker/internal/benthos/error/output_error.go b/worker/internal/benthos/error/output_error.go index 53e7b724f3..a8d2ce5d2a 100644 --- a/worker/internal/benthos/error/output_error.go +++ b/worker/internal/benthos/error/output_error.go @@ -13,6 +13,7 @@ func errorOutputSpec() *service.ConfigSpec { return service.NewConfigSpec(). Summary(`Sends stop Activity signal`). Field(service.NewStringField("error_msg")). + Field(service.NewIntField("max_retries").Optional()). Field(service.NewIntField("max_in_flight").Default(64)). Field(service.NewBatchPolicyField("batching")) } @@ -31,6 +32,7 @@ func RegisterErrorOutput(env *service.Environment, stopActivityChannel chan erro if err != nil { return nil, service.BatchPolicy{}, -1, err } + out, err := newErrorOutput(conf, mgr, stopActivityChannel) if err != nil { return nil, service.BatchPolicy{}, -1, err @@ -44,10 +46,19 @@ func newErrorOutput(conf *service.ParsedConfig, mgr *service.Resources, channel if err != nil { return nil, err } + var retries *int + if conf.Contains("max_retries") { + maxRetries, err := conf.FieldInt("max_retries") + if err != nil { + return nil, err + } + retries = &maxRetries + } return &errorOutput{ logger: mgr.Logger(), stopActivityChannel: channel, errorMsg: errMsg, + retries: retries, }, nil } @@ -55,6 +66,7 @@ type errorOutput struct { logger *service.Logger stopActivityChannel chan error errorMsg *service.InterpolatedString + retries *int } func (e *errorOutput) Connect(ctx context.Context) error { @@ -67,7 +79,10 @@ func (e *errorOutput) WriteBatch(ctx context.Context, batch service.MessageBatch if err != nil { return fmt.Errorf("error message interpolation error: %w", err) } - if neosync_benthos.IsMaxConnectionError(errMsg) { + if !neosync_benthos.ShouldTerminate(errMsg) || (e.retries != nil && *e.retries > 0) { + if e.retries != nil { + *e.retries-- + } // throw error so that benthos retries return errors.New(errMsg) } diff --git a/worker/internal/benthos/utils.go b/worker/internal/benthos/utils.go index 6f64bbed35..7787cf59b1 100644 --- a/worker/internal/benthos/utils.go +++ b/worker/internal/benthos/utils.go @@ -21,23 +21,21 @@ func ToSha256(input string) string { return fmt.Sprintf("%x", sha256.Sum256([]byte(input))) } -// checks if the error message matches a max connections error -func IsMaxConnectionError(errMsg string) bool { - // list of known error messages for when max connections are reached - maxConnErrors := []string{ +func containsIgnoreCase(s, substr string) bool { + return strings.Contains(strings.ToLower(s), strings.ToLower(substr)) +} + +// checks if the error should stop activity +func ShouldTerminate(errMsg string) bool { + // list of known error messages to terminate activity + stopErrors := []string{ "too many clients already", - "remaining connection slots are reserved", - "maximum number of connections reached", } - for _, errStr := range maxConnErrors { + for _, errStr := range stopErrors { if containsIgnoreCase(errMsg, errStr) { return true } } return false } - -func containsIgnoreCase(s, substr string) bool { - return strings.Contains(strings.ToLower(s), strings.ToLower(substr)) -} diff --git a/worker/internal/benthos/utils_test.go b/worker/internal/benthos/utils_test.go index bf3ce5f85e..ef0f353445 100644 --- a/worker/internal/benthos/utils_test.go +++ b/worker/internal/benthos/utils_test.go @@ -12,7 +12,7 @@ func Test_BuildBenthosTable(t *testing.T) { assert.Equal(t, BuildBenthosTable("", "users"), "users", "Handles an empty schema") } -func Test_IsMaxConnectionError(t *testing.T) { +func Test_ShouldTerminate(t *testing.T) { tests := []struct { name string errMsg string @@ -30,7 +30,7 @@ func Test_IsMaxConnectionError(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - actual := IsMaxConnectionError(tt.errMsg) + actual := ShouldTerminate(tt.errMsg) assert.Equal(t, tt.expected, actual) }) } diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index 8789d0c8eb..a1607df7bc 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -108,6 +108,7 @@ func (b *benthosBuilder) GenerateBenthosConfigs( // get depenendcy configs // split root tables vs children tables //////////////////////////////////////////////////////////////////////// + test := []any{} sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) if err != nil { @@ -363,6 +364,7 @@ func (b *benthosBuilder) GenerateBenthosConfigs( // processorConfigs = append(processorConfigs, *pc) // } + retries := 10 resp.Config.Output.Broker.Outputs = append(resp.Config.Output.Broker.Outputs, neosync_benthos.Outputs{ Fallback: []neosync_benthos.Outputs{ { @@ -398,7 +400,8 @@ func (b *benthosBuilder) GenerateBenthosConfigs( // kills activity depending on error // TODO add retry here {Error: &neosync_benthos.ErrorOutputConfig{ - ErrorMsg: `${! meta("fallback_error")}`, + ErrorMsg: `${! meta("fallback_error")}`, + MaxRetries: &retries, Batching: &neosync_benthos.Batching{ Period: "5s", Count: 100, From 5186c1518d478189292d39c9361bc9e0a6774f2d Mon Sep 17 00:00:00 2001 From: Alisha Date: Mon, 22 Apr 2024 10:54:47 -0700 Subject: [PATCH 09/12] clean up --- .../datasync/activities/gen-benthos-configs/benthos-builder.go | 1 - 1 file changed, 1 deletion(-) diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index 71619e98de..78d95c2b3d 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -108,7 +108,6 @@ func (b *benthosBuilder) GenerateBenthosConfigs( // get depenendcy configs // split root tables vs children tables //////////////////////////////////////////////////////////////////////// - test := []any{} sourceConnection, err := b.getJobSourceConnection(ctx, job.GetSource()) if err != nil { From 6f3ffcea57a0f4545b39c2bee367b31f3ef53653 Mon Sep 17 00:00:00 2001 From: Alisha Date: Mon, 22 Apr 2024 10:55:24 -0700 Subject: [PATCH 10/12] stash --- docs/protos/data/proto_docs.json | 12161 ++++++++++++++++ .../client/mgmt/v1alpha1/api_key_connect.ts | 76 + .../src/client/mgmt/v1alpha1/api_key_pb.ts | 491 + .../src/client/mgmt/v1alpha1/auth_connect.ts | 89 + .../sdk/src/client/mgmt/v1alpha1/auth_pb.ts | 559 + .../mgmt/v1alpha1/connection_connect.ts | 110 + .../mgmt/v1alpha1/connection_data_connect.ts | 90 + .../mgmt/v1alpha1/connection_data_pb.ts | 1171 ++ .../src/client/mgmt/v1alpha1/connection_pb.ts | 1575 ++ .../src/client/mgmt/v1alpha1/job_connect.ts | 265 + .../sdk/src/client/mgmt/v1alpha1/job_pb.ts | 4384 ++++++ .../client/mgmt/v1alpha1/metrics_connect.ts | 39 + .../src/client/mgmt/v1alpha1/metrics_pb.ts | 398 + .../mgmt/v1alpha1/transformer_connect.ts | 107 + .../client/mgmt/v1alpha1/transformer_pb.ts | 3180 ++++ .../mgmt/v1alpha1/user_account_connect.ts | 179 + .../client/mgmt/v1alpha1/user_account_pb.ts | 1659 +++ 17 files changed, 26533 insertions(+) create mode 100644 docs/protos/data/proto_docs.json create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts create mode 100644 frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts diff --git a/docs/protos/data/proto_docs.json b/docs/protos/data/proto_docs.json new file mode 100644 index 0000000000..1b0439c611 --- /dev/null +++ b/docs/protos/data/proto_docs.json @@ -0,0 +1,12161 @@ +{ + "files": [ + { + "name": "mgmt/v1alpha1/api_key.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": false, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [], + "extensions": [], + "messages": [ + { + "name": "AccountApiKey", + "longName": "AccountApiKey", + "fullName": "mgmt.v1alpha1.AccountApiKey", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "The friendly name of the API Key", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_by_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_by_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "key_value", + "description": "key_value is only returned on initial creation or when it is regenerated", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_key_value", + "defaultValue": "" + }, + { + "name": "user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "expires_at", + "description": "The timestamp of what the API key expires and will not longer be usable.", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateAccountApiKeyRequest", + "longName": "CreateAccountApiKeyRequest", + "fullName": "mgmt.v1alpha1.CreateAccountApiKeyRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "expires_at", + "description": "Validate between now and one year: now \u003c x \u003c 365 days", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateAccountApiKeyResponse", + "longName": "CreateAccountApiKeyResponse", + "fullName": "mgmt.v1alpha1.CreateAccountApiKeyResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "api_key", + "description": "", + "label": "", + "type": "AccountApiKey", + "longType": "AccountApiKey", + "fullType": "mgmt.v1alpha1.AccountApiKey", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteAccountApiKeyRequest", + "longName": "DeleteAccountApiKeyRequest", + "fullName": "mgmt.v1alpha1.DeleteAccountApiKeyRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteAccountApiKeyResponse", + "longName": "DeleteAccountApiKeyResponse", + "fullName": "mgmt.v1alpha1.DeleteAccountApiKeyResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetAccountApiKeyRequest", + "longName": "GetAccountApiKeyRequest", + "fullName": "mgmt.v1alpha1.GetAccountApiKeyRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountApiKeyResponse", + "longName": "GetAccountApiKeyResponse", + "fullName": "mgmt.v1alpha1.GetAccountApiKeyResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "api_key", + "description": "", + "label": "", + "type": "AccountApiKey", + "longType": "AccountApiKey", + "fullType": "mgmt.v1alpha1.AccountApiKey", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountApiKeysRequest", + "longName": "GetAccountApiKeysRequest", + "fullName": "mgmt.v1alpha1.GetAccountApiKeysRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountApiKeysResponse", + "longName": "GetAccountApiKeysResponse", + "fullName": "mgmt.v1alpha1.GetAccountApiKeysResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "api_keys", + "description": "", + "label": "repeated", + "type": "AccountApiKey", + "longType": "AccountApiKey", + "fullType": "mgmt.v1alpha1.AccountApiKey", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RegenerateAccountApiKeyRequest", + "longName": "RegenerateAccountApiKeyRequest", + "fullName": "mgmt.v1alpha1.RegenerateAccountApiKeyRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "expires_at", + "description": "Validate between now and one year: now \u003c x \u003c 365 days", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RegenerateAccountApiKeyResponse", + "longName": "RegenerateAccountApiKeyResponse", + "fullName": "mgmt.v1alpha1.RegenerateAccountApiKeyResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "api_key", + "description": "", + "label": "", + "type": "AccountApiKey", + "longType": "AccountApiKey", + "fullType": "mgmt.v1alpha1.AccountApiKey", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "ApiKeyService", + "longName": "ApiKeyService", + "fullName": "mgmt.v1alpha1.ApiKeyService", + "description": "Service that manages the lifecycle of API Keys that are associated with a specific Account.", + "methods": [ + { + "name": "GetAccountApiKeys", + "description": "Retrieves a list of Account API Keys", + "requestType": "GetAccountApiKeysRequest", + "requestLongType": "GetAccountApiKeysRequest", + "requestFullType": "mgmt.v1alpha1.GetAccountApiKeysRequest", + "requestStreaming": false, + "responseType": "GetAccountApiKeysResponse", + "responseLongType": "GetAccountApiKeysResponse", + "responseFullType": "mgmt.v1alpha1.GetAccountApiKeysResponse", + "responseStreaming": false + }, + { + "name": "GetAccountApiKey", + "description": "Retrieves a single API Key", + "requestType": "GetAccountApiKeyRequest", + "requestLongType": "GetAccountApiKeyRequest", + "requestFullType": "mgmt.v1alpha1.GetAccountApiKeyRequest", + "requestStreaming": false, + "responseType": "GetAccountApiKeyResponse", + "responseLongType": "GetAccountApiKeyResponse", + "responseFullType": "mgmt.v1alpha1.GetAccountApiKeyResponse", + "responseStreaming": false + }, + { + "name": "CreateAccountApiKey", + "description": "Creates a single API Key\nThis method will return the decrypted contents of the API key", + "requestType": "CreateAccountApiKeyRequest", + "requestLongType": "CreateAccountApiKeyRequest", + "requestFullType": "mgmt.v1alpha1.CreateAccountApiKeyRequest", + "requestStreaming": false, + "responseType": "CreateAccountApiKeyResponse", + "responseLongType": "CreateAccountApiKeyResponse", + "responseFullType": "mgmt.v1alpha1.CreateAccountApiKeyResponse", + "responseStreaming": false + }, + { + "name": "RegenerateAccountApiKey", + "description": "Regenerates a single API Key with a new expiration time\nThis method will return the decrypted contents of the API key", + "requestType": "RegenerateAccountApiKeyRequest", + "requestLongType": "RegenerateAccountApiKeyRequest", + "requestFullType": "mgmt.v1alpha1.RegenerateAccountApiKeyRequest", + "requestStreaming": false, + "responseType": "RegenerateAccountApiKeyResponse", + "responseLongType": "RegenerateAccountApiKeyResponse", + "responseFullType": "mgmt.v1alpha1.RegenerateAccountApiKeyResponse", + "responseStreaming": false + }, + { + "name": "DeleteAccountApiKey", + "description": "Deletes an API Key from the system.", + "requestType": "DeleteAccountApiKeyRequest", + "requestLongType": "DeleteAccountApiKeyRequest", + "requestFullType": "mgmt.v1alpha1.DeleteAccountApiKeyRequest", + "requestStreaming": false, + "responseType": "DeleteAccountApiKeyResponse", + "responseLongType": "DeleteAccountApiKeyResponse", + "responseFullType": "mgmt.v1alpha1.DeleteAccountApiKeyResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/auth.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": false, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [], + "extensions": [], + "messages": [ + { + "name": "AccessToken", + "longName": "AccessToken", + "fullName": "mgmt.v1alpha1.AccessToken", + "description": "A decoded representation of an Access token from the backing auth server", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "access_token", + "description": "The access token that will be provided in subsequent requests to provide authenticated access to the Api", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "refresh_token", + "description": "Token that can be used to retrieve a refreshed access token.\nWill not be provided if the offline_access scope is not provided in the initial login flow.", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_refresh_token", + "defaultValue": "" + }, + { + "name": "expires_in", + "description": "Relative time in seconds that the access token will expire. Combine with the current time to get the expires_at time.", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "scope", + "description": "The scopes that the access token have", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "id_token", + "description": "The identity token of the authenticated user", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_id_token", + "defaultValue": "" + }, + { + "name": "token_type", + "description": "The token type. For JWTs, this will be `Bearer`", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CheckTokenRequest", + "longName": "CheckTokenRequest", + "fullName": "mgmt.v1alpha1.CheckTokenRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "CheckTokenResponse", + "longName": "CheckTokenResponse", + "fullName": "mgmt.v1alpha1.CheckTokenResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetAuthStatusRequest", + "longName": "GetAuthStatusRequest", + "fullName": "mgmt.v1alpha1.GetAuthStatusRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetAuthStatusResponse", + "longName": "GetAuthStatusResponse", + "fullName": "mgmt.v1alpha1.GetAuthStatusResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "is_enabled", + "description": "Whether or not the server has authentication enabled.\nThis tells the client if it is expected to send access tokens.", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAuthorizeUrlRequest", + "longName": "GetAuthorizeUrlRequest", + "fullName": "mgmt.v1alpha1.GetAuthorizeUrlRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "state", + "description": "The state that's generated by the client that is passed along to prevent tampering", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "redirect_uri", + "description": "The redirect uri that the client will be redirected back to during the auth request", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "scope", + "description": "The scopes the client is requesting as a part of the oauth login request", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAuthorizeUrlResponse", + "longName": "GetAuthorizeUrlResponse", + "fullName": "mgmt.v1alpha1.GetAuthorizeUrlResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "url", + "description": "The generated url that is the client will be redirected to during the Oauth flow", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetCliIssuerRequest", + "longName": "GetCliIssuerRequest", + "fullName": "mgmt.v1alpha1.GetCliIssuerRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetCliIssuerResponse", + "longName": "GetCliIssuerResponse", + "fullName": "mgmt.v1alpha1.GetCliIssuerResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "issuer_url", + "description": "The backing authentication issuer url", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "audience", + "description": "The audience that will be used in the access token. This corresponds to the \"aud\" claim", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "LoginCliRequest", + "longName": "LoginCliRequest", + "fullName": "mgmt.v1alpha1.LoginCliRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "code", + "description": "The oauth code", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "redirect_uri", + "description": "The oauth redirect uri that the client uses during the oauth request", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "LoginCliResponse", + "longName": "LoginCliResponse", + "fullName": "mgmt.v1alpha1.LoginCliResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "access_token", + "description": "The access token that is returned on successful login", + "label": "", + "type": "AccessToken", + "longType": "AccessToken", + "fullType": "mgmt.v1alpha1.AccessToken", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RefreshCliRequest", + "longName": "RefreshCliRequest", + "fullName": "mgmt.v1alpha1.RefreshCliRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "refresh_token", + "description": "The token used to retrieve a new access token.", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RefreshCliResponse", + "longName": "RefreshCliResponse", + "fullName": "mgmt.v1alpha1.RefreshCliResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "access_token", + "description": "The access token that is returned on successful refresh", + "label": "", + "type": "AccessToken", + "longType": "AccessToken", + "fullType": "mgmt.v1alpha1.AccessToken", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "AuthService", + "longName": "AuthService", + "fullName": "mgmt.v1alpha1.AuthService", + "description": "Service that handles generic Authentication for Neosync\nToday this is mostly used by the CLI to receive authentication information", + "methods": [ + { + "name": "LoginCli", + "description": "Used by the CLI to login to Neosync with OAuth.", + "requestType": "LoginCliRequest", + "requestLongType": "LoginCliRequest", + "requestFullType": "mgmt.v1alpha1.LoginCliRequest", + "requestStreaming": false, + "responseType": "LoginCliResponse", + "responseLongType": "LoginCliResponse", + "responseFullType": "mgmt.v1alpha1.LoginCliResponse", + "responseStreaming": false + }, + { + "name": "RefreshCli", + "description": "Used by the CLI to refresh an expired Neosync accesss token.\nThis should only be used if an access token was previously retrieved from the `LoginCli` or `RefreshCli` methods.", + "requestType": "RefreshCliRequest", + "requestLongType": "RefreshCliRequest", + "requestFullType": "mgmt.v1alpha1.RefreshCliRequest", + "requestStreaming": false, + "responseType": "RefreshCliResponse", + "responseLongType": "RefreshCliResponse", + "responseFullType": "mgmt.v1alpha1.RefreshCliResponse", + "responseStreaming": false + }, + { + "name": "CheckToken", + "description": "Empty endpoint to simply check if the provided access token is valid", + "requestType": "CheckTokenRequest", + "requestLongType": "CheckTokenRequest", + "requestFullType": "mgmt.v1alpha1.CheckTokenRequest", + "requestStreaming": false, + "responseType": "CheckTokenResponse", + "responseLongType": "CheckTokenResponse", + "responseFullType": "mgmt.v1alpha1.CheckTokenResponse", + "responseStreaming": false + }, + { + "name": "GetCliIssuer", + "description": "Used by the CLI to retrieve Auth Issuer information\n@deprecated", + "requestType": "GetCliIssuerRequest", + "requestLongType": "GetCliIssuerRequest", + "requestFullType": "mgmt.v1alpha1.GetCliIssuerRequest", + "requestStreaming": false, + "responseType": "GetCliIssuerResponse", + "responseLongType": "GetCliIssuerResponse", + "responseFullType": "mgmt.v1alpha1.GetCliIssuerResponse", + "responseStreaming": false + }, + { + "name": "GetAuthorizeUrl", + "description": "Used by the CLI to retrieve an Authorize URL for use with OAuth login.", + "requestType": "GetAuthorizeUrlRequest", + "requestLongType": "GetAuthorizeUrlRequest", + "requestFullType": "mgmt.v1alpha1.GetAuthorizeUrlRequest", + "requestStreaming": false, + "responseType": "GetAuthorizeUrlResponse", + "responseLongType": "GetAuthorizeUrlResponse", + "responseFullType": "mgmt.v1alpha1.GetAuthorizeUrlResponse", + "responseStreaming": false + }, + { + "name": "GetAuthStatus", + "description": "Returns the auth status of the API server. Whether or not the backend has authentication enabled.\nThis is used by clients to make decisions on whether or not they should send access tokens to the API.", + "requestType": "GetAuthStatusRequest", + "requestLongType": "GetAuthStatusRequest", + "requestFullType": "mgmt.v1alpha1.GetAuthStatusRequest", + "requestStreaming": false, + "responseType": "GetAuthStatusResponse", + "responseLongType": "GetAuthStatusResponse", + "responseFullType": "mgmt.v1alpha1.GetAuthStatusResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/connection.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": false, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [], + "extensions": [], + "messages": [ + { + "name": "AwsS3ConnectionConfig", + "longName": "AwsS3ConnectionConfig", + "fullName": "mgmt.v1alpha1.AwsS3ConnectionConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "bucket_arn", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "", + "options": { + "deprecated": true + } + }, + { + "name": "path_prefix", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_path_prefix", + "defaultValue": "" + }, + { + "name": "credentials", + "description": "", + "label": "optional", + "type": "AwsS3Credentials", + "longType": "AwsS3Credentials", + "fullType": "mgmt.v1alpha1.AwsS3Credentials", + "ismap": false, + "isoneof": true, + "oneofdecl": "_credentials", + "defaultValue": "" + }, + { + "name": "region", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_region", + "defaultValue": "" + }, + { + "name": "endpoint", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_endpoint", + "defaultValue": "" + }, + { + "name": "bucket", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AwsS3Credentials", + "longName": "AwsS3Credentials", + "fullName": "mgmt.v1alpha1.AwsS3Credentials", + "description": "S3 Credentials that are used by the worker process.\nNote: this may be optionally provided if the worker that is being hosted has environment credentials to the S3 bucket instead.", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "profile", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_profile", + "defaultValue": "" + }, + { + "name": "access_key_id", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_access_key_id", + "defaultValue": "" + }, + { + "name": "secret_access_key", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_secret_access_key", + "defaultValue": "" + }, + { + "name": "session_token", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_session_token", + "defaultValue": "" + }, + { + "name": "from_ec2_role", + "description": "", + "label": "optional", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": true, + "oneofdecl": "_from_ec2_role", + "defaultValue": "" + }, + { + "name": "role_arn", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_role_arn", + "defaultValue": "" + }, + { + "name": "role_external_id", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_role_external_id", + "defaultValue": "" + } + ] + }, + { + "name": "CheckConnectionConfigRequest", + "longName": "CheckConnectionConfigRequest", + "fullName": "mgmt.v1alpha1.CheckConnectionConfigRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_config", + "description": "", + "label": "", + "type": "ConnectionConfig", + "longType": "ConnectionConfig", + "fullType": "mgmt.v1alpha1.ConnectionConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CheckConnectionConfigResponse", + "longName": "CheckConnectionConfigResponse", + "fullName": "mgmt.v1alpha1.CheckConnectionConfigResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "is_connected", + "description": "Whether or not the API was able to ping the connection", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_error", + "description": "This is the error that was received if the API was unable to connect", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_connection_error", + "defaultValue": "" + }, + { + "name": "privileges", + "description": "", + "label": "repeated", + "type": "ConnectionRolePrivilege", + "longType": "ConnectionRolePrivilege", + "fullType": "mgmt.v1alpha1.ConnectionRolePrivilege", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CheckSqlQueryRequest", + "longName": "CheckSqlQueryRequest", + "fullName": "mgmt.v1alpha1.CheckSqlQueryRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "The connection id that the query will be checked against", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "query", + "description": "The full query that will be run through a PREPARE statement", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CheckSqlQueryResponse", + "longName": "CheckSqlQueryResponse", + "fullName": "mgmt.v1alpha1.CheckSqlQueryResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "is_valid", + "description": "The query is run through PREPARE. Returns valid if it correctly compiled", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "erorr_message", + "description": "The error message returned by the sql client if the prepare did not return successfully", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_erorr_message", + "defaultValue": "" + } + ] + }, + { + "name": "Connection", + "longName": "Connection", + "fullName": "mgmt.v1alpha1.Connection", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_config", + "description": "", + "label": "", + "type": "ConnectionConfig", + "longType": "ConnectionConfig", + "fullType": "mgmt.v1alpha1.ConnectionConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_by_user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_by_user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ConnectionConfig", + "longName": "ConnectionConfig", + "fullName": "mgmt.v1alpha1.ConnectionConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "pg_config", + "description": "", + "label": "", + "type": "PostgresConnectionConfig", + "longType": "PostgresConnectionConfig", + "fullType": "mgmt.v1alpha1.PostgresConnectionConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "aws_s3_config", + "description": "", + "label": "", + "type": "AwsS3ConnectionConfig", + "longType": "AwsS3ConnectionConfig", + "fullType": "mgmt.v1alpha1.AwsS3ConnectionConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "mysql_config", + "description": "", + "label": "", + "type": "MysqlConnectionConfig", + "longType": "MysqlConnectionConfig", + "fullType": "mgmt.v1alpha1.MysqlConnectionConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "local_dir_config", + "description": "Configures a connection to a directory available on the local file system", + "label": "", + "type": "LocalDirectoryConnectionConfig", + "longType": "LocalDirectoryConnectionConfig", + "fullType": "mgmt.v1alpha1.LocalDirectoryConnectionConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "ConnectionRolePrivilege", + "longName": "ConnectionRolePrivilege", + "fullName": "mgmt.v1alpha1.ConnectionRolePrivilege", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "grantee", + "description": "The role that was given the permissions", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schema", + "description": "The database schema. Ex: public", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table", + "description": "The name of the table in the schema", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "privilege_type", + "description": "The privileges given to that role", + "label": "repeated", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateConnectionRequest", + "longName": "CreateConnectionRequest", + "fullName": "mgmt.v1alpha1.CreateConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "The friendly name of the connection", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_config", + "description": "", + "label": "", + "type": "ConnectionConfig", + "longType": "ConnectionConfig", + "fullType": "mgmt.v1alpha1.ConnectionConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateConnectionResponse", + "longName": "CreateConnectionResponse", + "fullName": "mgmt.v1alpha1.CreateConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection", + "description": "", + "label": "", + "type": "Connection", + "longType": "Connection", + "fullType": "mgmt.v1alpha1.Connection", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteConnectionRequest", + "longName": "DeleteConnectionRequest", + "fullName": "mgmt.v1alpha1.DeleteConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteConnectionResponse", + "longName": "DeleteConnectionResponse", + "fullName": "mgmt.v1alpha1.DeleteConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetConnectionRequest", + "longName": "GetConnectionRequest", + "fullName": "mgmt.v1alpha1.GetConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionResponse", + "longName": "GetConnectionResponse", + "fullName": "mgmt.v1alpha1.GetConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection", + "description": "", + "label": "", + "type": "Connection", + "longType": "Connection", + "fullType": "mgmt.v1alpha1.Connection", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionsRequest", + "longName": "GetConnectionsRequest", + "fullName": "mgmt.v1alpha1.GetConnectionsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionsResponse", + "longName": "GetConnectionsResponse", + "fullName": "mgmt.v1alpha1.GetConnectionsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connections", + "description": "", + "label": "repeated", + "type": "Connection", + "longType": "Connection", + "fullType": "mgmt.v1alpha1.Connection", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsConnectionNameAvailableRequest", + "longName": "IsConnectionNameAvailableRequest", + "fullName": "mgmt.v1alpha1.IsConnectionNameAvailableRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsConnectionNameAvailableResponse", + "longName": "IsConnectionNameAvailableResponse", + "fullName": "mgmt.v1alpha1.IsConnectionNameAvailableResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "is_available", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "LocalDirectoryConnectionConfig", + "longName": "LocalDirectoryConnectionConfig", + "fullName": "mgmt.v1alpha1.LocalDirectoryConnectionConfig", + "description": "Configures a connection to a directory available on the local file system", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "path", + "description": "The absolute path to a directory that is available on the local file system to the API and Worker nodes", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlConnection", + "longName": "MysqlConnection", + "fullName": "mgmt.v1alpha1.MysqlConnection", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "user", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "pass", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "protocol", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "host", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "port", + "description": "", + "label": "", + "type": "int32", + "longType": "int32", + "fullType": "int32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlConnectionConfig", + "longName": "MysqlConnectionConfig", + "fullName": "mgmt.v1alpha1.MysqlConnectionConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "url", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "connection_config", + "defaultValue": "" + }, + { + "name": "connection", + "description": "", + "label": "", + "type": "MysqlConnection", + "longType": "MysqlConnection", + "fullType": "mgmt.v1alpha1.MysqlConnection", + "ismap": false, + "isoneof": true, + "oneofdecl": "connection_config", + "defaultValue": "" + }, + { + "name": "tunnel", + "description": "Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet", + "label": "", + "type": "SSHTunnel", + "longType": "SSHTunnel", + "fullType": "mgmt.v1alpha1.SSHTunnel", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_options", + "description": "", + "label": "", + "type": "SqlConnectionOptions", + "longType": "SqlConnectionOptions", + "fullType": "mgmt.v1alpha1.SqlConnectionOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresConnection", + "longName": "PostgresConnection", + "fullName": "mgmt.v1alpha1.PostgresConnection", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "host", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "port", + "description": "", + "label": "", + "type": "int32", + "longType": "int32", + "fullType": "int32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "user", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "pass", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "ssl_mode", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_ssl_mode", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresConnectionConfig", + "longName": "PostgresConnectionConfig", + "fullName": "mgmt.v1alpha1.PostgresConnectionConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "url", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "connection_config", + "defaultValue": "" + }, + { + "name": "connection", + "description": "", + "label": "", + "type": "PostgresConnection", + "longType": "PostgresConnection", + "fullType": "mgmt.v1alpha1.PostgresConnection", + "ismap": false, + "isoneof": true, + "oneofdecl": "connection_config", + "defaultValue": "" + }, + { + "name": "tunnel", + "description": "Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet", + "label": "", + "type": "SSHTunnel", + "longType": "SSHTunnel", + "fullType": "mgmt.v1alpha1.SSHTunnel", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_options", + "description": "", + "label": "", + "type": "SqlConnectionOptions", + "longType": "SqlConnectionOptions", + "fullType": "mgmt.v1alpha1.SqlConnectionOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SSHAuthentication", + "longName": "SSHAuthentication", + "fullName": "mgmt.v1alpha1.SSHAuthentication", + "description": "SSH Authentication", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "passphrase", + "description": "", + "label": "", + "type": "SSHPassphrase", + "longType": "SSHPassphrase", + "fullType": "mgmt.v1alpha1.SSHPassphrase", + "ismap": false, + "isoneof": true, + "oneofdecl": "auth_config", + "defaultValue": "" + }, + { + "name": "private_key", + "description": "", + "label": "", + "type": "SSHPrivateKey", + "longType": "SSHPrivateKey", + "fullType": "mgmt.v1alpha1.SSHPrivateKey", + "ismap": false, + "isoneof": true, + "oneofdecl": "auth_config", + "defaultValue": "" + } + ] + }, + { + "name": "SSHPassphrase", + "longName": "SSHPassphrase", + "fullName": "mgmt.v1alpha1.SSHPassphrase", + "description": "Contains the configuration needed to retrieve the SSH passphrase for the tunnel", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "value", + "description": "The password to be used to connect to the SSH server\n\neventually we can expand this to allow pulling from other sources.", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SSHPrivateKey", + "longName": "SSHPrivateKey", + "fullName": "mgmt.v1alpha1.SSHPrivateKey", + "description": "Contains the configuration needed to retrieve the SSH private key for the tunnel", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "value", + "description": "The private key in PEM format.", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "passphrase", + "description": "If the private key is encrypted, this value should decrypt it.", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_passphrase", + "defaultValue": "" + } + ] + }, + { + "name": "SSHTunnel", + "longName": "SSHTunnel", + "fullName": "mgmt.v1alpha1.SSHTunnel", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "host", + "description": "The host of the SSH server", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "port", + "description": "The port of the SSH server, typically 22", + "label": "", + "type": "int32", + "longType": "int32", + "fullType": "int32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "user", + "description": "The user that will be used to authenticate", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "known_host_public_key", + "description": "Optionally provide the public key of the known host of the SSH tunnel that we are connecting to.\nIf this is not provided, the server will blindly connect to the host with the given credentials.\nThis is not recommended for production use!", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_known_host_public_key", + "defaultValue": "" + }, + { + "name": "authentication", + "description": "Provide the authentication required to successfully connect to the SSH server for tunneling", + "label": "", + "type": "SSHAuthentication", + "longType": "SSHAuthentication", + "fullType": "mgmt.v1alpha1.SSHAuthentication", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SqlConnectionOptions", + "longName": "SqlConnectionOptions", + "fullName": "mgmt.v1alpha1.SqlConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "max_connection_limit", + "description": "", + "label": "optional", + "type": "int32", + "longType": "int32", + "fullType": "int32", + "ismap": false, + "isoneof": true, + "oneofdecl": "_max_connection_limit", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateConnectionRequest", + "longName": "UpdateConnectionRequest", + "fullName": "mgmt.v1alpha1.UpdateConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_config", + "description": "", + "label": "", + "type": "ConnectionConfig", + "longType": "ConnectionConfig", + "fullType": "mgmt.v1alpha1.ConnectionConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateConnectionResponse", + "longName": "UpdateConnectionResponse", + "fullName": "mgmt.v1alpha1.UpdateConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection", + "description": "", + "label": "", + "type": "Connection", + "longType": "Connection", + "fullType": "mgmt.v1alpha1.Connection", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "ConnectionService", + "longName": "ConnectionService", + "fullName": "mgmt.v1alpha1.ConnectionService", + "description": "Service for managing datasource connections.\nThis is a primary data model in Neosync and is used in reference when hooking up Jobs to synchronize and generate data.", + "methods": [ + { + "name": "GetConnections", + "description": "Returns a list of connections associated with the account", + "requestType": "GetConnectionsRequest", + "requestLongType": "GetConnectionsRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionsRequest", + "requestStreaming": false, + "responseType": "GetConnectionsResponse", + "responseLongType": "GetConnectionsResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionsResponse", + "responseStreaming": false + }, + { + "name": "GetConnection", + "description": "Returns a single connection", + "requestType": "GetConnectionRequest", + "requestLongType": "GetConnectionRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionRequest", + "requestStreaming": false, + "responseType": "GetConnectionResponse", + "responseLongType": "GetConnectionResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionResponse", + "responseStreaming": false + }, + { + "name": "CreateConnection", + "description": "Creates a new connection", + "requestType": "CreateConnectionRequest", + "requestLongType": "CreateConnectionRequest", + "requestFullType": "mgmt.v1alpha1.CreateConnectionRequest", + "requestStreaming": false, + "responseType": "CreateConnectionResponse", + "responseLongType": "CreateConnectionResponse", + "responseFullType": "mgmt.v1alpha1.CreateConnectionResponse", + "responseStreaming": false + }, + { + "name": "UpdateConnection", + "description": "Updates an existing connection", + "requestType": "UpdateConnectionRequest", + "requestLongType": "UpdateConnectionRequest", + "requestFullType": "mgmt.v1alpha1.UpdateConnectionRequest", + "requestStreaming": false, + "responseType": "UpdateConnectionResponse", + "responseLongType": "UpdateConnectionResponse", + "responseFullType": "mgmt.v1alpha1.UpdateConnectionResponse", + "responseStreaming": false + }, + { + "name": "DeleteConnection", + "description": "Removes a connection from the system.", + "requestType": "DeleteConnectionRequest", + "requestLongType": "DeleteConnectionRequest", + "requestFullType": "mgmt.v1alpha1.DeleteConnectionRequest", + "requestStreaming": false, + "responseType": "DeleteConnectionResponse", + "responseLongType": "DeleteConnectionResponse", + "responseFullType": "mgmt.v1alpha1.DeleteConnectionResponse", + "responseStreaming": false + }, + { + "name": "IsConnectionNameAvailable", + "description": "Connections have friendly names, this method checks if the requested name is available in the system based on the account", + "requestType": "IsConnectionNameAvailableRequest", + "requestLongType": "IsConnectionNameAvailableRequest", + "requestFullType": "mgmt.v1alpha1.IsConnectionNameAvailableRequest", + "requestStreaming": false, + "responseType": "IsConnectionNameAvailableResponse", + "responseLongType": "IsConnectionNameAvailableResponse", + "responseFullType": "mgmt.v1alpha1.IsConnectionNameAvailableResponse", + "responseStreaming": false + }, + { + "name": "CheckConnectionConfig", + "description": "Checks if the connection config is connectable by the backend.\nUsed mostly to verify that a connection is valid prior to creating a Connection object.", + "requestType": "CheckConnectionConfigRequest", + "requestLongType": "CheckConnectionConfigRequest", + "requestFullType": "mgmt.v1alpha1.CheckConnectionConfigRequest", + "requestStreaming": false, + "responseType": "CheckConnectionConfigResponse", + "responseLongType": "CheckConnectionConfigResponse", + "responseFullType": "mgmt.v1alpha1.CheckConnectionConfigResponse", + "responseStreaming": false + }, + { + "name": "CheckSqlQuery", + "description": "Checks a constructed SQL query against a sql-based connection to see if it's valid based on that connection's data schema\nThis is useful when constructing subsets to see if the WHERE clause is correct", + "requestType": "CheckSqlQueryRequest", + "requestLongType": "CheckSqlQueryRequest", + "requestFullType": "mgmt.v1alpha1.CheckSqlQueryRequest", + "requestStreaming": false, + "responseType": "CheckSqlQueryResponse", + "responseLongType": "CheckSqlQueryResponse", + "responseFullType": "mgmt.v1alpha1.CheckSqlQueryResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/connection_data.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": false, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [], + "extensions": [], + "messages": [ + { + "name": "AwsS3SchemaConfig", + "longName": "AwsS3SchemaConfig", + "fullName": "mgmt.v1alpha1.AwsS3SchemaConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + }, + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + } + ] + }, + { + "name": "AwsS3StreamConfig", + "longName": "AwsS3StreamConfig", + "fullName": "mgmt.v1alpha1.AwsS3StreamConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + }, + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + } + ] + }, + { + "name": "ConnectionSchemaConfig", + "longName": "ConnectionSchemaConfig", + "fullName": "mgmt.v1alpha1.ConnectionSchemaConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "pg_config", + "description": "", + "label": "", + "type": "PostgresSchemaConfig", + "longType": "PostgresSchemaConfig", + "fullType": "mgmt.v1alpha1.PostgresSchemaConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "aws_s3_config", + "description": "", + "label": "", + "type": "AwsS3SchemaConfig", + "longType": "AwsS3SchemaConfig", + "fullType": "mgmt.v1alpha1.AwsS3SchemaConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "mysql_config", + "description": "", + "label": "", + "type": "MysqlSchemaConfig", + "longType": "MysqlSchemaConfig", + "fullType": "mgmt.v1alpha1.MysqlSchemaConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "ConnectionStreamConfig", + "longName": "ConnectionStreamConfig", + "fullName": "mgmt.v1alpha1.ConnectionStreamConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "pg_config", + "description": "", + "label": "", + "type": "PostgresStreamConfig", + "longType": "PostgresStreamConfig", + "fullType": "mgmt.v1alpha1.PostgresStreamConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "aws_s3_config", + "description": "", + "label": "", + "type": "AwsS3StreamConfig", + "longType": "AwsS3StreamConfig", + "fullType": "mgmt.v1alpha1.AwsS3StreamConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "mysql_config", + "description": "", + "label": "", + "type": "MysqlStreamConfig", + "longType": "MysqlStreamConfig", + "fullType": "mgmt.v1alpha1.MysqlStreamConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "DatabaseColumn", + "longName": "DatabaseColumn", + "fullName": "mgmt.v1alpha1.DatabaseColumn", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "The database schema. Ex: public", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table", + "description": "The name of the table in the schema", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "column", + "description": "The name of the column", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "data_type", + "description": "The datatype of the column", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "is_nullable", + "description": "The isNullable Flag of the column", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ForeignConstraint", + "longName": "ForeignConstraint", + "fullName": "mgmt.v1alpha1.ForeignConstraint", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "column", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "is_nullable", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "foreign_key", + "description": "", + "label": "", + "type": "ForeignKey", + "longType": "ForeignKey", + "fullType": "mgmt.v1alpha1.ForeignKey", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ForeignConstraintTables", + "longName": "ForeignConstraintTables", + "fullName": "mgmt.v1alpha1.ForeignConstraintTables", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "constraints", + "description": "", + "label": "repeated", + "type": "ForeignConstraint", + "longType": "ForeignConstraint", + "fullType": "mgmt.v1alpha1.ForeignConstraint", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ForeignKey", + "longName": "ForeignKey", + "fullName": "mgmt.v1alpha1.ForeignKey", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "column", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionDataStreamRequest", + "longName": "GetConnectionDataStreamRequest", + "fullName": "mgmt.v1alpha1.GetConnectionDataStreamRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "stream_config", + "description": "", + "label": "", + "type": "ConnectionStreamConfig", + "longType": "ConnectionStreamConfig", + "fullType": "mgmt.v1alpha1.ConnectionStreamConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionDataStreamResponse", + "longName": "GetConnectionDataStreamResponse", + "fullName": "mgmt.v1alpha1.GetConnectionDataStreamResponse", + "description": "Each stream response is a single row in the requested schema and table", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "row", + "description": "A map of column name to the bytes value of the data that was found for that column and row", + "label": "repeated", + "type": "RowEntry", + "longType": "GetConnectionDataStreamResponse.RowEntry", + "fullType": "mgmt.v1alpha1.GetConnectionDataStreamResponse.RowEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RowEntry", + "longName": "GetConnectionDataStreamResponse.RowEntry", + "fullName": "mgmt.v1alpha1.GetConnectionDataStreamResponse.RowEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "bytes", + "longType": "bytes", + "fullType": "bytes", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionForeignConstraintsRequest", + "longName": "GetConnectionForeignConstraintsRequest", + "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionForeignConstraintsResponse", + "longName": "GetConnectionForeignConstraintsResponse", + "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse", + "description": "Dependency constraints for a specific table", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table_constraints", + "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and the list of tables that it depends on, also `\u003cschema\u003e.\u003ctable\u003e` format.", + "label": "repeated", + "type": "TableConstraintsEntry", + "longType": "GetConnectionForeignConstraintsResponse.TableConstraintsEntry", + "fullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse.TableConstraintsEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TableConstraintsEntry", + "longName": "GetConnectionForeignConstraintsResponse.TableConstraintsEntry", + "fullName": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse.TableConstraintsEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "ForeignConstraintTables", + "longType": "ForeignConstraintTables", + "fullType": "mgmt.v1alpha1.ForeignConstraintTables", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionInitStatementsRequest", + "longName": "GetConnectionInitStatementsRequest", + "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "options", + "description": "", + "label": "", + "type": "InitStatementOptions", + "longType": "InitStatementOptions", + "fullType": "mgmt.v1alpha1.InitStatementOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionInitStatementsResponse", + "longName": "GetConnectionInitStatementsResponse", + "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse", + "description": "Init statement for a specific table", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table_init_statements", + "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the table init statement.", + "label": "repeated", + "type": "TableInitStatementsEntry", + "longType": "GetConnectionInitStatementsResponse.TableInitStatementsEntry", + "fullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableInitStatementsEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table_truncate_statements", + "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the table truncate statement.", + "label": "repeated", + "type": "TableTruncateStatementsEntry", + "longType": "GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", + "fullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TableInitStatementsEntry", + "longName": "GetConnectionInitStatementsResponse.TableInitStatementsEntry", + "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableInitStatementsEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TableTruncateStatementsEntry", + "longName": "GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", + "fullName": "mgmt.v1alpha1.GetConnectionInitStatementsResponse.TableTruncateStatementsEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionPrimaryConstraintsRequest", + "longName": "GetConnectionPrimaryConstraintsRequest", + "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest", + "description": "Primary constraints for a specific table", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionPrimaryConstraintsResponse", + "longName": "GetConnectionPrimaryConstraintsResponse", + "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table_constraints", + "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the primary constraint", + "label": "repeated", + "type": "TableConstraintsEntry", + "longType": "GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", + "fullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TableConstraintsEntry", + "longName": "GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", + "fullName": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse.TableConstraintsEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "PrimaryConstraint", + "longType": "PrimaryConstraint", + "fullType": "mgmt.v1alpha1.PrimaryConstraint", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionSchemaRequest", + "longName": "GetConnectionSchemaRequest", + "fullName": "mgmt.v1alpha1.GetConnectionSchemaRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schema_config", + "description": "", + "label": "", + "type": "ConnectionSchemaConfig", + "longType": "ConnectionSchemaConfig", + "fullType": "mgmt.v1alpha1.ConnectionSchemaConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionSchemaResponse", + "longName": "GetConnectionSchemaResponse", + "fullName": "mgmt.v1alpha1.GetConnectionSchemaResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schemas", + "description": "", + "label": "repeated", + "type": "DatabaseColumn", + "longType": "DatabaseColumn", + "fullType": "mgmt.v1alpha1.DatabaseColumn", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionUniqueConstraintsRequest", + "longName": "GetConnectionUniqueConstraintsRequest", + "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest", + "description": "Unique constraints for a specific table", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetConnectionUniqueConstraintsResponse", + "longName": "GetConnectionUniqueConstraintsResponse", + "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table_constraints", + "description": "the key here is \u003cschema\u003e.\u003ctable\u003e and value is the unique constraint", + "label": "repeated", + "type": "TableConstraintsEntry", + "longType": "GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", + "fullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", + "ismap": true, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TableConstraintsEntry", + "longName": "GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", + "fullName": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse.TableConstraintsEntry", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "key", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "value", + "description": "", + "label": "", + "type": "UniqueConstraint", + "longType": "UniqueConstraint", + "fullType": "mgmt.v1alpha1.UniqueConstraint", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "InitStatementOptions", + "longName": "InitStatementOptions", + "fullName": "mgmt.v1alpha1.InitStatementOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "init_schema", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "truncate_before_insert", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "truncate_cascade", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlSchemaConfig", + "longName": "MysqlSchemaConfig", + "fullName": "mgmt.v1alpha1.MysqlSchemaConfig", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "MysqlStreamConfig", + "longName": "MysqlStreamConfig", + "fullName": "mgmt.v1alpha1.MysqlStreamConfig", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "PostgresSchemaConfig", + "longName": "PostgresSchemaConfig", + "fullName": "mgmt.v1alpha1.PostgresSchemaConfig", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "PostgresStreamConfig", + "longName": "PostgresStreamConfig", + "fullName": "mgmt.v1alpha1.PostgresStreamConfig", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "PrimaryConstraint", + "longName": "PrimaryConstraint", + "fullName": "mgmt.v1alpha1.PrimaryConstraint", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "columns", + "description": "", + "label": "repeated", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UniqueConstraint", + "longName": "UniqueConstraint", + "fullName": "mgmt.v1alpha1.UniqueConstraint", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "columns", + "description": "", + "label": "repeated", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "ConnectionDataService", + "longName": "ConnectionDataService", + "fullName": "mgmt.v1alpha1.ConnectionDataService", + "description": "Service for managing connection data.\nThis is used in handle data from a connection", + "methods": [ + { + "name": "GetConnectionDataStream", + "description": "Streaming endpoint that will stream the data available from the Connection to the client.\nUsed primarily by the CLI sync command.", + "requestType": "GetConnectionDataStreamRequest", + "requestLongType": "GetConnectionDataStreamRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionDataStreamRequest", + "requestStreaming": false, + "responseType": "GetConnectionDataStreamResponse", + "responseLongType": "GetConnectionDataStreamResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionDataStreamResponse", + "responseStreaming": true + }, + { + "name": "GetConnectionSchema", + "description": "Returns the schema for a specific connection. Used mostly for SQL-based connections", + "requestType": "GetConnectionSchemaRequest", + "requestLongType": "GetConnectionSchemaRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionSchemaRequest", + "requestStreaming": false, + "responseType": "GetConnectionSchemaResponse", + "responseLongType": "GetConnectionSchemaResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionSchemaResponse", + "responseStreaming": false + }, + { + "name": "GetConnectionForeignConstraints", + "description": "For a specific connection, returns the foreign key constraints. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to determine stream order.", + "requestType": "GetConnectionForeignConstraintsRequest", + "requestLongType": "GetConnectionForeignConstraintsRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest", + "requestStreaming": false, + "responseType": "GetConnectionForeignConstraintsResponse", + "responseLongType": "GetConnectionForeignConstraintsResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse", + "responseStreaming": false + }, + { + "name": "GetConnectionPrimaryConstraints", + "description": "For a specific connection, returns the primary key constraints. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to determine stream order.", + "requestType": "GetConnectionPrimaryConstraintsRequest", + "requestLongType": "GetConnectionPrimaryConstraintsRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest", + "requestStreaming": false, + "responseType": "GetConnectionPrimaryConstraintsResponse", + "responseLongType": "GetConnectionPrimaryConstraintsResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse", + "responseStreaming": false + }, + { + "name": "GetConnectionInitStatements", + "description": "For a specific connection, returns the init table statements. Mostly useful for SQL-based Connections.\nUsed primarily by the CLI sync command to create table schema init statement.", + "requestType": "GetConnectionInitStatementsRequest", + "requestLongType": "GetConnectionInitStatementsRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionInitStatementsRequest", + "requestStreaming": false, + "responseType": "GetConnectionInitStatementsResponse", + "responseLongType": "GetConnectionInitStatementsResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionInitStatementsResponse", + "responseStreaming": false + }, + { + "name": "GetConnectionUniqueConstraints", + "description": "For a specific connection, returns the unique constraints. Mostly useful for SQL-based connections.", + "requestType": "GetConnectionUniqueConstraintsRequest", + "requestLongType": "GetConnectionUniqueConstraintsRequest", + "requestFullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest", + "requestStreaming": false, + "responseType": "GetConnectionUniqueConstraintsResponse", + "responseLongType": "GetConnectionUniqueConstraintsResponse", + "responseFullType": "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/transformer.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": true, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [ + { + "name": "SupportedJobType", + "longName": "SupportedJobType", + "fullName": "mgmt.v1alpha1.SupportedJobType", + "description": "", + "values": [ + { + "name": "SUPPORTED_JOB_TYPE_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "SUPPORTED_JOB_TYPE_SYNC", + "number": "1", + "description": "" + }, + { + "name": "SUPPORTED_JOB_TYPE_GENERATE", + "number": "2", + "description": "" + } + ] + }, + { + "name": "TransformerDataType", + "longName": "TransformerDataType", + "fullName": "mgmt.v1alpha1.TransformerDataType", + "description": "", + "values": [ + { + "name": "TRANSFORMER_DATA_TYPE_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_STRING", + "number": "1", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_INT64", + "number": "2", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_BOOLEAN", + "number": "3", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_FLOAT64", + "number": "4", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_NULL", + "number": "5", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_ANY", + "number": "6", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_TIME", + "number": "7", + "description": "" + }, + { + "name": "TRANSFORMER_DATA_TYPE_UUID", + "number": "8", + "description": "" + } + ] + }, + { + "name": "TransformerSource", + "longName": "TransformerSource", + "fullName": "mgmt.v1alpha1.TransformerSource", + "description": "", + "values": [ + { + "name": "TRANSFORMER_SOURCE_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_PASSTHROUGH", + "number": "1", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_DEFAULT", + "number": "2", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT", + "number": "3", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_EMAIL", + "number": "4", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_EMAIL", + "number": "5", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_BOOL", + "number": "6", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER", + "number": "7", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_CITY", + "number": "8", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER", + "number": "9", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_FIRST_NAME", + "number": "10", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_FLOAT64", + "number": "11", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS", + "number": "12", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_FULL_NAME", + "number": "13", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_GENDER", + "number": "14", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER", + "number": "15", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_INT64", + "number": "16", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64", + "number": "17", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_LAST_NAME", + "number": "18", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_SHA256HASH", + "number": "19", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_SSN", + "number": "20", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_STATE", + "number": "21", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS", + "number": "22", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER", + "number": "23", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_STRING", + "number": "24", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING", + "number": "25", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP", + "number": "26", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_USERNAME", + "number": "27", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP", + "number": "28", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_UUID", + "number": "29", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_ZIPCODE", + "number": "30", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER", + "number": "31", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME", + "number": "32", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_FLOAT64", + "number": "33", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME", + "number": "34", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER", + "number": "35", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_INT64", + "number": "36", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME", + "number": "37", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER", + "number": "38", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_STRING", + "number": "39", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_NULL", + "number": "40", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_CATEGORICAL", + "number": "42", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE", + "number": "43", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_USER_DEFINED", + "number": "44", + "description": "" + }, + { + "name": "TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT", + "number": "45", + "description": "" + } + ] + } + ], + "extensions": [], + "messages": [ + { + "name": "CreateUserDefinedTransformerRequest", + "longName": "CreateUserDefinedTransformerRequest", + "fullName": "mgmt.v1alpha1.CreateUserDefinedTransformerRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "description", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "type", + "description": "@deprecated", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "TransformerSource", + "longType": "TransformerSource", + "fullType": "mgmt.v1alpha1.TransformerSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "transformer_config", + "description": "", + "label": "", + "type": "TransformerConfig", + "longType": "TransformerConfig", + "fullType": "mgmt.v1alpha1.TransformerConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateUserDefinedTransformerResponse", + "longName": "CreateUserDefinedTransformerResponse", + "fullName": "mgmt.v1alpha1.CreateUserDefinedTransformerResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer", + "description": "", + "label": "", + "type": "UserDefinedTransformer", + "longType": "UserDefinedTransformer", + "fullType": "mgmt.v1alpha1.UserDefinedTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteUserDefinedTransformerRequest", + "longName": "DeleteUserDefinedTransformerRequest", + "fullName": "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteUserDefinedTransformerResponse", + "longName": "DeleteUserDefinedTransformerResponse", + "fullName": "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateBool", + "longName": "GenerateBool", + "fullName": "mgmt.v1alpha1.GenerateBool", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateCardNumber", + "longName": "GenerateCardNumber", + "fullName": "mgmt.v1alpha1.GenerateCardNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "valid_luhn", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateCategorical", + "longName": "GenerateCategorical", + "fullName": "mgmt.v1alpha1.GenerateCategorical", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "categories", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateCity", + "longName": "GenerateCity", + "fullName": "mgmt.v1alpha1.GenerateCity", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateDefault", + "longName": "GenerateDefault", + "fullName": "mgmt.v1alpha1.GenerateDefault", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateE164PhoneNumber", + "longName": "GenerateE164PhoneNumber", + "fullName": "mgmt.v1alpha1.GenerateE164PhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "min", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateEmail", + "longName": "GenerateEmail", + "fullName": "mgmt.v1alpha1.GenerateEmail", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateFirstName", + "longName": "GenerateFirstName", + "fullName": "mgmt.v1alpha1.GenerateFirstName", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateFloat64", + "longName": "GenerateFloat64", + "fullName": "mgmt.v1alpha1.GenerateFloat64", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "randomize_sign", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "min", + "description": "", + "label": "", + "type": "double", + "longType": "double", + "fullType": "double", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max", + "description": "", + "label": "", + "type": "double", + "longType": "double", + "fullType": "double", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "precision", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateFullAddress", + "longName": "GenerateFullAddress", + "fullName": "mgmt.v1alpha1.GenerateFullAddress", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateFullName", + "longName": "GenerateFullName", + "fullName": "mgmt.v1alpha1.GenerateFullName", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateGender", + "longName": "GenerateGender", + "fullName": "mgmt.v1alpha1.GenerateGender", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "abbreviate", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateInt64", + "longName": "GenerateInt64", + "fullName": "mgmt.v1alpha1.GenerateInt64", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "randomize_sign", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "min", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateInt64PhoneNumber", + "longName": "GenerateInt64PhoneNumber", + "fullName": "mgmt.v1alpha1.GenerateInt64PhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateJavascript", + "longName": "GenerateJavascript", + "fullName": "mgmt.v1alpha1.GenerateJavascript", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "code", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateLastName", + "longName": "GenerateLastName", + "fullName": "mgmt.v1alpha1.GenerateLastName", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateSSN", + "longName": "GenerateSSN", + "fullName": "mgmt.v1alpha1.GenerateSSN", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateSha256Hash", + "longName": "GenerateSha256Hash", + "fullName": "mgmt.v1alpha1.GenerateSha256Hash", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateState", + "longName": "GenerateState", + "fullName": "mgmt.v1alpha1.GenerateState", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateStreetAddress", + "longName": "GenerateStreetAddress", + "fullName": "mgmt.v1alpha1.GenerateStreetAddress", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateString", + "longName": "GenerateString", + "fullName": "mgmt.v1alpha1.GenerateString", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "min", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateStringPhoneNumber", + "longName": "GenerateStringPhoneNumber", + "fullName": "mgmt.v1alpha1.GenerateStringPhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "min", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateUnixTimestamp", + "longName": "GenerateUnixTimestamp", + "fullName": "mgmt.v1alpha1.GenerateUnixTimestamp", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateUsername", + "longName": "GenerateUsername", + "fullName": "mgmt.v1alpha1.GenerateUsername", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateUtcTimestamp", + "longName": "GenerateUtcTimestamp", + "fullName": "mgmt.v1alpha1.GenerateUtcTimestamp", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateUuid", + "longName": "GenerateUuid", + "fullName": "mgmt.v1alpha1.GenerateUuid", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "include_hyphens", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateZipcode", + "longName": "GenerateZipcode", + "fullName": "mgmt.v1alpha1.GenerateZipcode", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetSystemTransformerBySourceRequest", + "longName": "GetSystemTransformerBySourceRequest", + "fullName": "mgmt.v1alpha1.GetSystemTransformerBySourceRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "source", + "description": "", + "label": "", + "type": "TransformerSource", + "longType": "TransformerSource", + "fullType": "mgmt.v1alpha1.TransformerSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetSystemTransformerBySourceResponse", + "longName": "GetSystemTransformerBySourceResponse", + "fullName": "mgmt.v1alpha1.GetSystemTransformerBySourceResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer", + "description": "", + "label": "", + "type": "SystemTransformer", + "longType": "SystemTransformer", + "fullType": "mgmt.v1alpha1.SystemTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetSystemTransformersRequest", + "longName": "GetSystemTransformersRequest", + "fullName": "mgmt.v1alpha1.GetSystemTransformersRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetSystemTransformersResponse", + "longName": "GetSystemTransformersResponse", + "fullName": "mgmt.v1alpha1.GetSystemTransformersResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformers", + "description": "", + "label": "repeated", + "type": "SystemTransformer", + "longType": "SystemTransformer", + "fullType": "mgmt.v1alpha1.SystemTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserDefinedTransformerByIdRequest", + "longName": "GetUserDefinedTransformerByIdRequest", + "fullName": "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserDefinedTransformerByIdResponse", + "longName": "GetUserDefinedTransformerByIdResponse", + "fullName": "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer", + "description": "", + "label": "", + "type": "UserDefinedTransformer", + "longType": "UserDefinedTransformer", + "fullType": "mgmt.v1alpha1.UserDefinedTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserDefinedTransformersRequest", + "longName": "GetUserDefinedTransformersRequest", + "fullName": "mgmt.v1alpha1.GetUserDefinedTransformersRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserDefinedTransformersResponse", + "longName": "GetUserDefinedTransformersResponse", + "fullName": "mgmt.v1alpha1.GetUserDefinedTransformersResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformers", + "description": "", + "label": "repeated", + "type": "UserDefinedTransformer", + "longType": "UserDefinedTransformer", + "fullType": "mgmt.v1alpha1.UserDefinedTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsTransformerNameAvailableRequest", + "longName": "IsTransformerNameAvailableRequest", + "fullName": "mgmt.v1alpha1.IsTransformerNameAvailableRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "transformer_name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsTransformerNameAvailableResponse", + "longName": "IsTransformerNameAvailableResponse", + "fullName": "mgmt.v1alpha1.IsTransformerNameAvailableResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "is_available", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "Null", + "longName": "Null", + "fullName": "mgmt.v1alpha1.Null", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "Passthrough", + "longName": "Passthrough", + "fullName": "mgmt.v1alpha1.Passthrough", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "SystemTransformer", + "longName": "SystemTransformer", + "fullName": "mgmt.v1alpha1.SystemTransformer", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "description", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "data_type", + "description": "", + "label": "", + "type": "TransformerDataType", + "longType": "TransformerDataType", + "fullType": "mgmt.v1alpha1.TransformerDataType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "TransformerSource", + "longType": "TransformerSource", + "fullType": "mgmt.v1alpha1.TransformerSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "config", + "description": "", + "label": "", + "type": "TransformerConfig", + "longType": "TransformerConfig", + "fullType": "mgmt.v1alpha1.TransformerConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "data_types", + "description": "", + "label": "repeated", + "type": "TransformerDataType", + "longType": "TransformerDataType", + "fullType": "mgmt.v1alpha1.TransformerDataType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "supported_job_types", + "description": "", + "label": "repeated", + "type": "SupportedJobType", + "longType": "SupportedJobType", + "fullType": "mgmt.v1alpha1.SupportedJobType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformCharacterScramble", + "longName": "TransformCharacterScramble", + "fullName": "mgmt.v1alpha1.TransformCharacterScramble", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "user_provided_regex", + "description": "a user provided regular expression that they wish to validate if it compiles in GO", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_user_provided_regex", + "defaultValue": "" + } + ] + }, + { + "name": "TransformE164PhoneNumber", + "longName": "TransformE164PhoneNumber", + "fullName": "mgmt.v1alpha1.TransformE164PhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformEmail", + "longName": "TransformEmail", + "fullName": "mgmt.v1alpha1.TransformEmail", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_domain", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "excluded_domains", + "description": "", + "label": "repeated", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformFirstName", + "longName": "TransformFirstName", + "fullName": "mgmt.v1alpha1.TransformFirstName", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformFloat64", + "longName": "TransformFloat64", + "fullName": "mgmt.v1alpha1.TransformFloat64", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "randomization_range_min", + "description": "", + "label": "", + "type": "double", + "longType": "double", + "fullType": "double", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "randomization_range_max", + "description": "", + "label": "", + "type": "double", + "longType": "double", + "fullType": "double", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformFullName", + "longName": "TransformFullName", + "fullName": "mgmt.v1alpha1.TransformFullName", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformInt64", + "longName": "TransformInt64", + "fullName": "mgmt.v1alpha1.TransformInt64", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "randomization_range_min", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "randomization_range_max", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformInt64PhoneNumber", + "longName": "TransformInt64PhoneNumber", + "fullName": "mgmt.v1alpha1.TransformInt64PhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformJavascript", + "longName": "TransformJavascript", + "fullName": "mgmt.v1alpha1.TransformJavascript", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "code", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformLastName", + "longName": "TransformLastName", + "fullName": "mgmt.v1alpha1.TransformLastName", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformPhoneNumber", + "longName": "TransformPhoneNumber", + "fullName": "mgmt.v1alpha1.TransformPhoneNumber", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformString", + "longName": "TransformString", + "fullName": "mgmt.v1alpha1.TransformString", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "preserve_length", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TransformerConfig", + "longName": "TransformerConfig", + "fullName": "mgmt.v1alpha1.TransformerConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "generate_email_config", + "description": "", + "label": "", + "type": "GenerateEmail", + "longType": "GenerateEmail", + "fullType": "mgmt.v1alpha1.GenerateEmail", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_email_config", + "description": "", + "label": "", + "type": "TransformEmail", + "longType": "TransformEmail", + "fullType": "mgmt.v1alpha1.TransformEmail", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_bool_config", + "description": "", + "label": "", + "type": "GenerateBool", + "longType": "GenerateBool", + "fullType": "mgmt.v1alpha1.GenerateBool", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_card_number_config", + "description": "", + "label": "", + "type": "GenerateCardNumber", + "longType": "GenerateCardNumber", + "fullType": "mgmt.v1alpha1.GenerateCardNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_city_config", + "description": "", + "label": "", + "type": "GenerateCity", + "longType": "GenerateCity", + "fullType": "mgmt.v1alpha1.GenerateCity", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_e164_phone_number_config", + "description": "", + "label": "", + "type": "GenerateE164PhoneNumber", + "longType": "GenerateE164PhoneNumber", + "fullType": "mgmt.v1alpha1.GenerateE164PhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_first_name_config", + "description": "", + "label": "", + "type": "GenerateFirstName", + "longType": "GenerateFirstName", + "fullType": "mgmt.v1alpha1.GenerateFirstName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_float64_config", + "description": "", + "label": "", + "type": "GenerateFloat64", + "longType": "GenerateFloat64", + "fullType": "mgmt.v1alpha1.GenerateFloat64", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_full_address_config", + "description": "", + "label": "", + "type": "GenerateFullAddress", + "longType": "GenerateFullAddress", + "fullType": "mgmt.v1alpha1.GenerateFullAddress", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_full_name_config", + "description": "", + "label": "", + "type": "GenerateFullName", + "longType": "GenerateFullName", + "fullType": "mgmt.v1alpha1.GenerateFullName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_gender_config", + "description": "", + "label": "", + "type": "GenerateGender", + "longType": "GenerateGender", + "fullType": "mgmt.v1alpha1.GenerateGender", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_int64_phone_number_config", + "description": "", + "label": "", + "type": "GenerateInt64PhoneNumber", + "longType": "GenerateInt64PhoneNumber", + "fullType": "mgmt.v1alpha1.GenerateInt64PhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_int64_config", + "description": "", + "label": "", + "type": "GenerateInt64", + "longType": "GenerateInt64", + "fullType": "mgmt.v1alpha1.GenerateInt64", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_last_name_config", + "description": "", + "label": "", + "type": "GenerateLastName", + "longType": "GenerateLastName", + "fullType": "mgmt.v1alpha1.GenerateLastName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_sha256hash_config", + "description": "", + "label": "", + "type": "GenerateSha256Hash", + "longType": "GenerateSha256Hash", + "fullType": "mgmt.v1alpha1.GenerateSha256Hash", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_ssn_config", + "description": "", + "label": "", + "type": "GenerateSSN", + "longType": "GenerateSSN", + "fullType": "mgmt.v1alpha1.GenerateSSN", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_state_config", + "description": "", + "label": "", + "type": "GenerateState", + "longType": "GenerateState", + "fullType": "mgmt.v1alpha1.GenerateState", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_street_address_config", + "description": "", + "label": "", + "type": "GenerateStreetAddress", + "longType": "GenerateStreetAddress", + "fullType": "mgmt.v1alpha1.GenerateStreetAddress", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_string_phone_number_config", + "description": "", + "label": "", + "type": "GenerateStringPhoneNumber", + "longType": "GenerateStringPhoneNumber", + "fullType": "mgmt.v1alpha1.GenerateStringPhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_string_config", + "description": "", + "label": "", + "type": "GenerateString", + "longType": "GenerateString", + "fullType": "mgmt.v1alpha1.GenerateString", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_unixtimestamp_config", + "description": "", + "label": "", + "type": "GenerateUnixTimestamp", + "longType": "GenerateUnixTimestamp", + "fullType": "mgmt.v1alpha1.GenerateUnixTimestamp", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_username_config", + "description": "", + "label": "", + "type": "GenerateUsername", + "longType": "GenerateUsername", + "fullType": "mgmt.v1alpha1.GenerateUsername", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_utctimestamp_config", + "description": "", + "label": "", + "type": "GenerateUtcTimestamp", + "longType": "GenerateUtcTimestamp", + "fullType": "mgmt.v1alpha1.GenerateUtcTimestamp", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_uuid_config", + "description": "", + "label": "", + "type": "GenerateUuid", + "longType": "GenerateUuid", + "fullType": "mgmt.v1alpha1.GenerateUuid", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_zipcode_config", + "description": "", + "label": "", + "type": "GenerateZipcode", + "longType": "GenerateZipcode", + "fullType": "mgmt.v1alpha1.GenerateZipcode", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_e164_phone_number_config", + "description": "", + "label": "", + "type": "TransformE164PhoneNumber", + "longType": "TransformE164PhoneNumber", + "fullType": "mgmt.v1alpha1.TransformE164PhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_first_name_config", + "description": "", + "label": "", + "type": "TransformFirstName", + "longType": "TransformFirstName", + "fullType": "mgmt.v1alpha1.TransformFirstName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_float64_config", + "description": "", + "label": "", + "type": "TransformFloat64", + "longType": "TransformFloat64", + "fullType": "mgmt.v1alpha1.TransformFloat64", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_full_name_config", + "description": "", + "label": "", + "type": "TransformFullName", + "longType": "TransformFullName", + "fullType": "mgmt.v1alpha1.TransformFullName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_int64_phone_number_config", + "description": "", + "label": "", + "type": "TransformInt64PhoneNumber", + "longType": "TransformInt64PhoneNumber", + "fullType": "mgmt.v1alpha1.TransformInt64PhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_int64_config", + "description": "", + "label": "", + "type": "TransformInt64", + "longType": "TransformInt64", + "fullType": "mgmt.v1alpha1.TransformInt64", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_last_name_config", + "description": "", + "label": "", + "type": "TransformLastName", + "longType": "TransformLastName", + "fullType": "mgmt.v1alpha1.TransformLastName", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_phone_number_config", + "description": "", + "label": "", + "type": "TransformPhoneNumber", + "longType": "TransformPhoneNumber", + "fullType": "mgmt.v1alpha1.TransformPhoneNumber", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_string_config", + "description": "", + "label": "", + "type": "TransformString", + "longType": "TransformString", + "fullType": "mgmt.v1alpha1.TransformString", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "passthrough_config", + "description": "", + "label": "", + "type": "Passthrough", + "longType": "Passthrough", + "fullType": "mgmt.v1alpha1.Passthrough", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "nullconfig", + "description": "", + "label": "", + "type": "Null", + "longType": "Null", + "fullType": "mgmt.v1alpha1.Null", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "user_defined_transformer_config", + "description": "", + "label": "", + "type": "UserDefinedTransformerConfig", + "longType": "UserDefinedTransformerConfig", + "fullType": "mgmt.v1alpha1.UserDefinedTransformerConfig", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_default_config", + "description": "", + "label": "", + "type": "GenerateDefault", + "longType": "GenerateDefault", + "fullType": "mgmt.v1alpha1.GenerateDefault", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_javascript_config", + "description": "", + "label": "", + "type": "TransformJavascript", + "longType": "TransformJavascript", + "fullType": "mgmt.v1alpha1.TransformJavascript", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_categorical_config", + "description": "", + "label": "", + "type": "GenerateCategorical", + "longType": "GenerateCategorical", + "fullType": "mgmt.v1alpha1.GenerateCategorical", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "transform_character_scramble_config", + "description": "", + "label": "", + "type": "TransformCharacterScramble", + "longType": "TransformCharacterScramble", + "fullType": "mgmt.v1alpha1.TransformCharacterScramble", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate_javascript_config", + "description": "", + "label": "", + "type": "GenerateJavascript", + "longType": "GenerateJavascript", + "fullType": "mgmt.v1alpha1.GenerateJavascript", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateUserDefinedTransformerRequest", + "longName": "UpdateUserDefinedTransformerRequest", + "fullName": "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "description", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "transformer_config", + "description": "", + "label": "", + "type": "TransformerConfig", + "longType": "TransformerConfig", + "fullType": "mgmt.v1alpha1.TransformerConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateUserDefinedTransformerResponse", + "longName": "UpdateUserDefinedTransformerResponse", + "fullName": "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "transformer", + "description": "", + "label": "", + "type": "UserDefinedTransformer", + "longType": "UserDefinedTransformer", + "fullType": "mgmt.v1alpha1.UserDefinedTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UserDefinedTransformer", + "longName": "UserDefinedTransformer", + "fullName": "mgmt.v1alpha1.UserDefinedTransformer", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "description", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "data_type", + "description": "This property is readonly and is calculated based off the origin system transformer", + "label": "", + "type": "TransformerDataType", + "longType": "TransformerDataType", + "fullType": "mgmt.v1alpha1.TransformerDataType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "TransformerSource", + "longType": "TransformerSource", + "fullType": "mgmt.v1alpha1.TransformerSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "config", + "description": "", + "label": "", + "type": "TransformerConfig", + "longType": "TransformerConfig", + "fullType": "mgmt.v1alpha1.TransformerConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "data_types", + "description": "", + "label": "repeated", + "type": "TransformerDataType", + "longType": "TransformerDataType", + "fullType": "mgmt.v1alpha1.TransformerDataType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UserDefinedTransformerConfig", + "longName": "UserDefinedTransformerConfig", + "fullName": "mgmt.v1alpha1.UserDefinedTransformerConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ValidateUserJavascriptCodeRequest", + "longName": "ValidateUserJavascriptCodeRequest", + "fullName": "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "code", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ValidateUserJavascriptCodeResponse", + "longName": "ValidateUserJavascriptCodeResponse", + "fullName": "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "valid", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ValidateUserRegexCodeRequest", + "longName": "ValidateUserRegexCodeRequest", + "fullName": "mgmt.v1alpha1.ValidateUserRegexCodeRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "user_provided_regex", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ValidateUserRegexCodeResponse", + "longName": "ValidateUserRegexCodeResponse", + "fullName": "mgmt.v1alpha1.ValidateUserRegexCodeResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "valid", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "TransformersService", + "longName": "TransformersService", + "fullName": "mgmt.v1alpha1.TransformersService", + "description": "", + "methods": [ + { + "name": "GetSystemTransformers", + "description": "", + "requestType": "GetSystemTransformersRequest", + "requestLongType": "GetSystemTransformersRequest", + "requestFullType": "mgmt.v1alpha1.GetSystemTransformersRequest", + "requestStreaming": false, + "responseType": "GetSystemTransformersResponse", + "responseLongType": "GetSystemTransformersResponse", + "responseFullType": "mgmt.v1alpha1.GetSystemTransformersResponse", + "responseStreaming": false + }, + { + "name": "GetSystemTransformerBySource", + "description": "", + "requestType": "GetSystemTransformerBySourceRequest", + "requestLongType": "GetSystemTransformerBySourceRequest", + "requestFullType": "mgmt.v1alpha1.GetSystemTransformerBySourceRequest", + "requestStreaming": false, + "responseType": "GetSystemTransformerBySourceResponse", + "responseLongType": "GetSystemTransformerBySourceResponse", + "responseFullType": "mgmt.v1alpha1.GetSystemTransformerBySourceResponse", + "responseStreaming": false + }, + { + "name": "GetUserDefinedTransformers", + "description": "", + "requestType": "GetUserDefinedTransformersRequest", + "requestLongType": "GetUserDefinedTransformersRequest", + "requestFullType": "mgmt.v1alpha1.GetUserDefinedTransformersRequest", + "requestStreaming": false, + "responseType": "GetUserDefinedTransformersResponse", + "responseLongType": "GetUserDefinedTransformersResponse", + "responseFullType": "mgmt.v1alpha1.GetUserDefinedTransformersResponse", + "responseStreaming": false + }, + { + "name": "GetUserDefinedTransformerById", + "description": "", + "requestType": "GetUserDefinedTransformerByIdRequest", + "requestLongType": "GetUserDefinedTransformerByIdRequest", + "requestFullType": "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest", + "requestStreaming": false, + "responseType": "GetUserDefinedTransformerByIdResponse", + "responseLongType": "GetUserDefinedTransformerByIdResponse", + "responseFullType": "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse", + "responseStreaming": false + }, + { + "name": "CreateUserDefinedTransformer", + "description": "", + "requestType": "CreateUserDefinedTransformerRequest", + "requestLongType": "CreateUserDefinedTransformerRequest", + "requestFullType": "mgmt.v1alpha1.CreateUserDefinedTransformerRequest", + "requestStreaming": false, + "responseType": "CreateUserDefinedTransformerResponse", + "responseLongType": "CreateUserDefinedTransformerResponse", + "responseFullType": "mgmt.v1alpha1.CreateUserDefinedTransformerResponse", + "responseStreaming": false + }, + { + "name": "DeleteUserDefinedTransformer", + "description": "", + "requestType": "DeleteUserDefinedTransformerRequest", + "requestLongType": "DeleteUserDefinedTransformerRequest", + "requestFullType": "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest", + "requestStreaming": false, + "responseType": "DeleteUserDefinedTransformerResponse", + "responseLongType": "DeleteUserDefinedTransformerResponse", + "responseFullType": "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse", + "responseStreaming": false + }, + { + "name": "UpdateUserDefinedTransformer", + "description": "", + "requestType": "UpdateUserDefinedTransformerRequest", + "requestLongType": "UpdateUserDefinedTransformerRequest", + "requestFullType": "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest", + "requestStreaming": false, + "responseType": "UpdateUserDefinedTransformerResponse", + "responseLongType": "UpdateUserDefinedTransformerResponse", + "responseFullType": "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse", + "responseStreaming": false + }, + { + "name": "IsTransformerNameAvailable", + "description": "", + "requestType": "IsTransformerNameAvailableRequest", + "requestLongType": "IsTransformerNameAvailableRequest", + "requestFullType": "mgmt.v1alpha1.IsTransformerNameAvailableRequest", + "requestStreaming": false, + "responseType": "IsTransformerNameAvailableResponse", + "responseLongType": "IsTransformerNameAvailableResponse", + "responseFullType": "mgmt.v1alpha1.IsTransformerNameAvailableResponse", + "responseStreaming": false + }, + { + "name": "ValidateUserJavascriptCode", + "description": "", + "requestType": "ValidateUserJavascriptCodeRequest", + "requestLongType": "ValidateUserJavascriptCodeRequest", + "requestFullType": "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest", + "requestStreaming": false, + "responseType": "ValidateUserJavascriptCodeResponse", + "responseLongType": "ValidateUserJavascriptCodeResponse", + "responseFullType": "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse", + "responseStreaming": false + }, + { + "name": "ValidateUserRegexCode", + "description": "", + "requestType": "ValidateUserRegexCodeRequest", + "requestLongType": "ValidateUserRegexCodeRequest", + "requestFullType": "mgmt.v1alpha1.ValidateUserRegexCodeRequest", + "requestStreaming": false, + "responseType": "ValidateUserRegexCodeResponse", + "responseLongType": "ValidateUserRegexCodeResponse", + "responseFullType": "mgmt.v1alpha1.ValidateUserRegexCodeResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/job.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": true, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [ + { + "name": "ActivityStatus", + "longName": "ActivityStatus", + "fullName": "mgmt.v1alpha1.ActivityStatus", + "description": "", + "values": [ + { + "name": "ACTIVITY_STATUS_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "ACTIVITY_STATUS_SCHEDULED", + "number": "1", + "description": "" + }, + { + "name": "ACTIVITY_STATUS_STARTED", + "number": "2", + "description": "" + }, + { + "name": "ACTIVITY_STATUS_CANCELED", + "number": "3", + "description": "" + }, + { + "name": "ACTIVITY_STATUS_FAILED", + "number": "4", + "description": "" + } + ] + }, + { + "name": "JobRunStatus", + "longName": "JobRunStatus", + "fullName": "mgmt.v1alpha1.JobRunStatus", + "description": "An enumeration of job run statuses.", + "values": [ + { + "name": "JOB_RUN_STATUS_UNSPECIFIED", + "number": "0", + "description": "if the job run status is unknown" + }, + { + "name": "JOB_RUN_STATUS_PENDING", + "number": "1", + "description": "the run is pending and has not started yet" + }, + { + "name": "JOB_RUN_STATUS_RUNNING", + "number": "2", + "description": "the run is currently in progress" + }, + { + "name": "JOB_RUN_STATUS_COMPLETE", + "number": "3", + "description": "the run has successfully completed" + }, + { + "name": "JOB_RUN_STATUS_ERROR", + "number": "4", + "description": "the run ended with an error" + }, + { + "name": "JOB_RUN_STATUS_CANCELED", + "number": "5", + "description": "the run was cancelled" + }, + { + "name": "JOB_RUN_STATUS_TERMINATED", + "number": "6", + "description": "the run was terminated" + }, + { + "name": "JOB_RUN_STATUS_FAILED", + "number": "7", + "description": "the run ended in failure" + }, + { + "name": "JOB_RUN_STATUS_TIMED_OUT", + "number": "8", + "description": "the run was ended pre-maturely due to timeout" + } + ] + }, + { + "name": "JobStatus", + "longName": "JobStatus", + "fullName": "mgmt.v1alpha1.JobStatus", + "description": "", + "values": [ + { + "name": "JOB_STATUS_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "JOB_STATUS_ENABLED", + "number": "1", + "description": "" + }, + { + "name": "JOB_STATUS_PAUSED", + "number": "3", + "description": "" + }, + { + "name": "JOB_STATUS_DISABLED", + "number": "4", + "description": "" + } + ] + }, + { + "name": "LogLevel", + "longName": "LogLevel", + "fullName": "mgmt.v1alpha1.LogLevel", + "description": "", + "values": [ + { + "name": "LOG_LEVEL_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "LOG_LEVEL_DEBUG", + "number": "1", + "description": "" + }, + { + "name": "LOG_LEVEL_INFO", + "number": "2", + "description": "" + }, + { + "name": "LOG_LEVEL_WARN", + "number": "3", + "description": "" + }, + { + "name": "LOG_LEVEL_ERROR", + "number": "4", + "description": "" + } + ] + }, + { + "name": "LogWindow", + "longName": "LogWindow", + "fullName": "mgmt.v1alpha1.LogWindow", + "description": "", + "values": [ + { + "name": "LOG_WINDOW_NO_TIME_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "LOG_WINDOW_FIFTEEN_MIN", + "number": "1", + "description": "" + }, + { + "name": "LOG_WINDOW_ONE_HOUR", + "number": "2", + "description": "" + }, + { + "name": "LOG_WINDOW_ONE_DAY", + "number": "3", + "description": "" + } + ] + } + ], + "extensions": [], + "messages": [ + { + "name": "ActivityFailure", + "longName": "ActivityFailure", + "fullName": "mgmt.v1alpha1.ActivityFailure", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "message", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ActivityOptions", + "longName": "ActivityOptions", + "fullName": "mgmt.v1alpha1.ActivityOptions", + "description": "Config that contains various timeouts that are configured in the underlying temporal workflow(s) and activities", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "schedule_to_close_timeout", + "description": "Total time that a workflow is willing to wait for an activity to complete, including retries.\nMeasured in seconds", + "label": "optional", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": true, + "oneofdecl": "_schedule_to_close_timeout", + "defaultValue": "" + }, + { + "name": "start_to_close_timeout", + "description": "Max time of a single Temporal Activity execution attempt.\nThis timeout should be as short as the longest psosible execution of any activity (e.g. table sync).\nImportant to know that this is per retry attempt. Defaults to the schedule to close timeout if not provided.\nMeasured in seconds", + "label": "optional", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": true, + "oneofdecl": "_start_to_close_timeout", + "defaultValue": "" + }, + { + "name": "retry_policy", + "description": "Optionally define a retry policy for the activity\nIf max attempts is not set, the activity will retry indefinitely until the start to close timeout lapses", + "label": "", + "type": "RetryPolicy", + "longType": "RetryPolicy", + "fullType": "mgmt.v1alpha1.RetryPolicy", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AwsS3DestinationConnectionOptions", + "longName": "AwsS3DestinationConnectionOptions", + "fullName": "mgmt.v1alpha1.AwsS3DestinationConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "AwsS3SourceConnectionOptions", + "longName": "AwsS3SourceConnectionOptions", + "fullName": "mgmt.v1alpha1.AwsS3SourceConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CancelJobRunRequest", + "longName": "CancelJobRunRequest", + "fullName": "mgmt.v1alpha1.CancelJobRunRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CancelJobRunResponse", + "longName": "CancelJobRunResponse", + "fullName": "mgmt.v1alpha1.CancelJobRunResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "CreateJobDestination", + "longName": "CreateJobDestination", + "fullName": "mgmt.v1alpha1.CreateJobDestination", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "options", + "description": "", + "label": "", + "type": "JobDestinationOptions", + "longType": "JobDestinationOptions", + "fullType": "mgmt.v1alpha1.JobDestinationOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobDestinationConnectionsRequest", + "longName": "CreateJobDestinationConnectionsRequest", + "fullName": "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "destinations", + "description": "", + "label": "repeated", + "type": "CreateJobDestination", + "longType": "CreateJobDestination", + "fullType": "mgmt.v1alpha1.CreateJobDestination", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobDestinationConnectionsResponse", + "longName": "CreateJobDestinationConnectionsResponse", + "fullName": "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobRequest", + "longName": "CreateJobRequest", + "fullName": "mgmt.v1alpha1.CreateJobRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "The unique account identifier that this job will be associated with", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "job_name", + "description": "The unique, friendly name of the job. This is unique per account", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "cron_schedule", + "description": "Optionally provide a cron schedule. Goes into effect if the job status is set to enabled", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_cron_schedule", + "defaultValue": "" + }, + { + "name": "mappings", + "description": "", + "label": "repeated", + "type": "JobMapping", + "longType": "JobMapping", + "fullType": "mgmt.v1alpha1.JobMapping", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "JobSource", + "longType": "JobSource", + "fullType": "mgmt.v1alpha1.JobSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "destinations", + "description": "", + "label": "repeated", + "type": "CreateJobDestination", + "longType": "CreateJobDestination", + "fullType": "mgmt.v1alpha1.CreateJobDestination", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "initiate_job_run", + "description": "Initially trigger a run of this job regardless of its status or cron schedule", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "workflow_options", + "description": "Specify timeouts and other workflow options for the underlying temporal workflow", + "label": "", + "type": "WorkflowOptions", + "longType": "WorkflowOptions", + "fullType": "mgmt.v1alpha1.WorkflowOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "sync_options", + "description": "Specify timeout and retry options for data synchronization activities\nData sync activities are any piece of work that involves actually synchronizing data from a source to a destination\nFor the data sync and generate jobs, this will be applied per table", + "label": "", + "type": "ActivityOptions", + "longType": "ActivityOptions", + "fullType": "mgmt.v1alpha1.ActivityOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobResponse", + "longName": "CreateJobResponse", + "fullName": "mgmt.v1alpha1.CreateJobResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobRunRequest", + "longName": "CreateJobRunRequest", + "fullName": "mgmt.v1alpha1.CreateJobRunRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateJobRunResponse", + "longName": "CreateJobRunResponse", + "fullName": "mgmt.v1alpha1.CreateJobRunResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "DeleteJobDestinationConnectionRequest", + "longName": "DeleteJobDestinationConnectionRequest", + "fullName": "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "destination_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteJobDestinationConnectionResponse", + "longName": "DeleteJobDestinationConnectionResponse", + "fullName": "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "DeleteJobRequest", + "longName": "DeleteJobRequest", + "fullName": "mgmt.v1alpha1.DeleteJobRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteJobResponse", + "longName": "DeleteJobResponse", + "fullName": "mgmt.v1alpha1.DeleteJobResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "DeleteJobRunRequest", + "longName": "DeleteJobRunRequest", + "fullName": "mgmt.v1alpha1.DeleteJobRunRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DeleteJobRunResponse", + "longName": "DeleteJobRunResponse", + "fullName": "mgmt.v1alpha1.DeleteJobRunResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GenerateSourceOptions", + "longName": "GenerateSourceOptions", + "fullName": "mgmt.v1alpha1.GenerateSourceOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "schemas", + "description": "", + "label": "repeated", + "type": "GenerateSourceSchemaOption", + "longType": "GenerateSourceSchemaOption", + "fullType": "mgmt.v1alpha1.GenerateSourceSchemaOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "fk_source_connection_id", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_fk_source_connection_id", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateSourceSchemaOption", + "longName": "GenerateSourceSchemaOption", + "fullName": "mgmt.v1alpha1.GenerateSourceSchemaOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "tables", + "description": "", + "label": "repeated", + "type": "GenerateSourceTableOption", + "longType": "GenerateSourceTableOption", + "fullType": "mgmt.v1alpha1.GenerateSourceTableOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GenerateSourceTableOption", + "longName": "GenerateSourceTableOption", + "fullName": "mgmt.v1alpha1.GenerateSourceTableOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "row_count", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobNextRunsRequest", + "longName": "GetJobNextRunsRequest", + "fullName": "mgmt.v1alpha1.GetJobNextRunsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobNextRunsResponse", + "longName": "GetJobNextRunsResponse", + "fullName": "mgmt.v1alpha1.GetJobNextRunsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "next_runs", + "description": "", + "label": "", + "type": "JobNextRuns", + "longType": "JobNextRuns", + "fullType": "mgmt.v1alpha1.JobNextRuns", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRecentRunsRequest", + "longName": "GetJobRecentRunsRequest", + "fullName": "mgmt.v1alpha1.GetJobRecentRunsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRecentRunsResponse", + "longName": "GetJobRecentRunsResponse", + "fullName": "mgmt.v1alpha1.GetJobRecentRunsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "recent_runs", + "description": "", + "label": "repeated", + "type": "JobRecentRun", + "longType": "JobRecentRun", + "fullType": "mgmt.v1alpha1.JobRecentRun", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRequest", + "longName": "GetJobRequest", + "fullName": "mgmt.v1alpha1.GetJobRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobResponse", + "longName": "GetJobResponse", + "fullName": "mgmt.v1alpha1.GetJobResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunEventsRequest", + "longName": "GetJobRunEventsRequest", + "fullName": "mgmt.v1alpha1.GetJobRunEventsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunEventsResponse", + "longName": "GetJobRunEventsResponse", + "fullName": "mgmt.v1alpha1.GetJobRunEventsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "events", + "description": "", + "label": "repeated", + "type": "JobRunEvent", + "longType": "JobRunEvent", + "fullType": "mgmt.v1alpha1.JobRunEvent", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "is_run_complete", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunLogsStreamRequest", + "longName": "GetJobRunLogsStreamRequest", + "fullName": "mgmt.v1alpha1.GetJobRunLogsStreamRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "window", + "description": "The time window in which to retrieve the logs", + "label": "", + "type": "LogWindow", + "longType": "LogWindow", + "fullType": "mgmt.v1alpha1.LogWindow", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "should_tail", + "description": "Whether or not to tail the stream. Note: only works with k8s-pods and is not currently supported with Loki logs", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "max_log_lines", + "description": "Optionally provide a max log limit", + "label": "optional", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": true, + "oneofdecl": "_max_log_lines", + "defaultValue": "" + }, + { + "name": "log_levels", + "description": "Provide a list of log levels to filter by. If any of these are UNSPECIFIED, all log levels are returned.", + "label": "repeated", + "type": "LogLevel", + "longType": "LogLevel", + "fullType": "mgmt.v1alpha1.LogLevel", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunLogsStreamResponse", + "longName": "GetJobRunLogsStreamResponse", + "fullName": "mgmt.v1alpha1.GetJobRunLogsStreamResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "log_line", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "timestamp", + "description": "", + "label": "optional", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": true, + "oneofdecl": "_timestamp", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunRequest", + "longName": "GetJobRunRequest", + "fullName": "mgmt.v1alpha1.GetJobRunRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunResponse", + "longName": "GetJobRunResponse", + "fullName": "mgmt.v1alpha1.GetJobRunResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run", + "description": "", + "label": "", + "type": "JobRun", + "longType": "JobRun", + "fullType": "mgmt.v1alpha1.JobRun", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunsRequest", + "longName": "GetJobRunsRequest", + "fullName": "mgmt.v1alpha1.GetJobRunsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "id", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobRunsResponse", + "longName": "GetJobRunsResponse", + "fullName": "mgmt.v1alpha1.GetJobRunsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_runs", + "description": "", + "label": "repeated", + "type": "JobRun", + "longType": "JobRun", + "fullType": "mgmt.v1alpha1.JobRun", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobStatusRequest", + "longName": "GetJobStatusRequest", + "fullName": "mgmt.v1alpha1.GetJobStatusRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobStatusResponse", + "longName": "GetJobStatusResponse", + "fullName": "mgmt.v1alpha1.GetJobStatusResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "status", + "description": "", + "label": "", + "type": "JobStatus", + "longType": "JobStatus", + "fullType": "mgmt.v1alpha1.JobStatus", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobStatusesRequest", + "longName": "GetJobStatusesRequest", + "fullName": "mgmt.v1alpha1.GetJobStatusesRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobStatusesResponse", + "longName": "GetJobStatusesResponse", + "fullName": "mgmt.v1alpha1.GetJobStatusesResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "statuses", + "description": "", + "label": "repeated", + "type": "JobStatusRecord", + "longType": "JobStatusRecord", + "fullType": "mgmt.v1alpha1.JobStatusRecord", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobsRequest", + "longName": "GetJobsRequest", + "fullName": "mgmt.v1alpha1.GetJobsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetJobsResponse", + "longName": "GetJobsResponse", + "fullName": "mgmt.v1alpha1.GetJobsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "jobs", + "description": "", + "label": "repeated", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsJobNameAvailableRequest", + "longName": "IsJobNameAvailableRequest", + "fullName": "mgmt.v1alpha1.IsJobNameAvailableRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsJobNameAvailableResponse", + "longName": "IsJobNameAvailableResponse", + "fullName": "mgmt.v1alpha1.IsJobNameAvailableResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "is_available", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "Job", + "longName": "Job", + "fullName": "mgmt.v1alpha1.Job", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "The unique identifier of the job", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_by_user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_by_user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "The unique, friendly name of the job", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "JobSource", + "longType": "JobSource", + "fullType": "mgmt.v1alpha1.JobSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "destinations", + "description": "", + "label": "repeated", + "type": "JobDestination", + "longType": "JobDestination", + "fullType": "mgmt.v1alpha1.JobDestination", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "mappings", + "description": "", + "label": "repeated", + "type": "JobMapping", + "longType": "JobMapping", + "fullType": "mgmt.v1alpha1.JobMapping", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "cron_schedule", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_cron_schedule", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "The account identifier that a job is associated with", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "sync_options", + "description": "Specify timeout and retry options for data synchronization activities\nData sync activities are any piece of work that involves actually synchronizing data from a source to a destination\nFor the data sync and generate jobs, this will be applied per table", + "label": "", + "type": "ActivityOptions", + "longType": "ActivityOptions", + "fullType": "mgmt.v1alpha1.ActivityOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "workflow_options", + "description": "Specify timeouts and other workflow options for the underlying temporal workflow", + "label": "", + "type": "WorkflowOptions", + "longType": "WorkflowOptions", + "fullType": "mgmt.v1alpha1.WorkflowOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobDestination", + "longName": "JobDestination", + "fullName": "mgmt.v1alpha1.JobDestination", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "options", + "description": "", + "label": "", + "type": "JobDestinationOptions", + "longType": "JobDestinationOptions", + "fullType": "mgmt.v1alpha1.JobDestinationOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobDestinationOptions", + "longName": "JobDestinationOptions", + "fullName": "mgmt.v1alpha1.JobDestinationOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "postgres_options", + "description": "", + "label": "", + "type": "PostgresDestinationConnectionOptions", + "longType": "PostgresDestinationConnectionOptions", + "fullType": "mgmt.v1alpha1.PostgresDestinationConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "aws_s3_options", + "description": "", + "label": "", + "type": "AwsS3DestinationConnectionOptions", + "longType": "AwsS3DestinationConnectionOptions", + "fullType": "mgmt.v1alpha1.AwsS3DestinationConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "mysql_options", + "description": "", + "label": "", + "type": "MysqlDestinationConnectionOptions", + "longType": "MysqlDestinationConnectionOptions", + "fullType": "mgmt.v1alpha1.MysqlDestinationConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "JobMapping", + "longName": "JobMapping", + "fullName": "mgmt.v1alpha1.JobMapping", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "column", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "transformer", + "description": "", + "label": "", + "type": "JobMappingTransformer", + "longType": "JobMappingTransformer", + "fullType": "mgmt.v1alpha1.JobMappingTransformer", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobMappingTransformer", + "longName": "JobMappingTransformer", + "fullName": "mgmt.v1alpha1.JobMappingTransformer", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "source", + "description": "", + "label": "", + "type": "TransformerSource", + "longType": "TransformerSource", + "fullType": "mgmt.v1alpha1.TransformerSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "config", + "description": "", + "label": "", + "type": "TransformerConfig", + "longType": "TransformerConfig", + "fullType": "mgmt.v1alpha1.TransformerConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobNextRuns", + "longName": "JobNextRuns", + "fullName": "mgmt.v1alpha1.JobNextRuns", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "next_run_times", + "description": "", + "label": "repeated", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRecentRun", + "longName": "JobRecentRun", + "fullName": "mgmt.v1alpha1.JobRecentRun", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "start_time", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRun", + "longName": "JobRun", + "fullName": "mgmt.v1alpha1.JobRun", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "The id of the job run. This will currently be equivalent to the temporal workflow id", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "job_id", + "description": "The unique identifier of the job id this run is associated with", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "The name of the job run.", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "status", + "description": "the status of the job run", + "label": "", + "type": "JobRunStatus", + "longType": "JobRunStatus", + "fullType": "mgmt.v1alpha1.JobRunStatus", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "started_at", + "description": "A timestamp of when the run started", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "completed_at", + "description": "Available if the run completed or has not yet been archived by the system", + "label": "optional", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": true, + "oneofdecl": "_completed_at", + "defaultValue": "" + }, + { + "name": "pending_activities", + "description": "Pending activities are only returned when retrieving a specific job run and will not be returned when requesting job runs in list format", + "label": "repeated", + "type": "PendingActivity", + "longType": "PendingActivity", + "fullType": "mgmt.v1alpha1.PendingActivity", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRunEvent", + "longName": "JobRunEvent", + "fullName": "mgmt.v1alpha1.JobRunEvent", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "type", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "start_time", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "close_time", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "metadata", + "description": "", + "label": "", + "type": "JobRunEventMetadata", + "longType": "JobRunEventMetadata", + "fullType": "mgmt.v1alpha1.JobRunEventMetadata", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "tasks", + "description": "", + "label": "repeated", + "type": "JobRunEventTask", + "longType": "JobRunEventTask", + "fullType": "mgmt.v1alpha1.JobRunEventTask", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRunEventMetadata", + "longName": "JobRunEventMetadata", + "fullName": "mgmt.v1alpha1.JobRunEventMetadata", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "sync_metadata", + "description": "", + "label": "", + "type": "JobRunSyncMetadata", + "longType": "JobRunSyncMetadata", + "fullType": "mgmt.v1alpha1.JobRunSyncMetadata", + "ismap": false, + "isoneof": true, + "oneofdecl": "metadata", + "defaultValue": "" + } + ] + }, + { + "name": "JobRunEventTask", + "longName": "JobRunEventTask", + "fullName": "mgmt.v1alpha1.JobRunEventTask", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "type", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "event_time", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "error", + "description": "", + "label": "", + "type": "JobRunEventTaskError", + "longType": "JobRunEventTaskError", + "fullType": "mgmt.v1alpha1.JobRunEventTaskError", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRunEventTaskError", + "longName": "JobRunEventTaskError", + "fullName": "mgmt.v1alpha1.JobRunEventTaskError", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "message", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "retry_state", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobRunSyncMetadata", + "longName": "JobRunSyncMetadata", + "fullName": "mgmt.v1alpha1.JobRunSyncMetadata", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobSource", + "longName": "JobSource", + "fullName": "mgmt.v1alpha1.JobSource", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "options", + "description": "", + "label": "", + "type": "JobSourceOptions", + "longType": "JobSourceOptions", + "fullType": "mgmt.v1alpha1.JobSourceOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "JobSourceOptions", + "longName": "JobSourceOptions", + "fullName": "mgmt.v1alpha1.JobSourceOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "postgres", + "description": "", + "label": "", + "type": "PostgresSourceConnectionOptions", + "longType": "PostgresSourceConnectionOptions", + "fullType": "mgmt.v1alpha1.PostgresSourceConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "aws_s3", + "description": "", + "label": "", + "type": "AwsS3SourceConnectionOptions", + "longType": "AwsS3SourceConnectionOptions", + "fullType": "mgmt.v1alpha1.AwsS3SourceConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "mysql", + "description": "", + "label": "", + "type": "MysqlSourceConnectionOptions", + "longType": "MysqlSourceConnectionOptions", + "fullType": "mgmt.v1alpha1.MysqlSourceConnectionOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + }, + { + "name": "generate", + "description": "", + "label": "", + "type": "GenerateSourceOptions", + "longType": "GenerateSourceOptions", + "fullType": "mgmt.v1alpha1.GenerateSourceOptions", + "ismap": false, + "isoneof": true, + "oneofdecl": "config", + "defaultValue": "" + } + ] + }, + { + "name": "JobSourceSqlSubetSchemas", + "longName": "JobSourceSqlSubetSchemas", + "fullName": "mgmt.v1alpha1.JobSourceSqlSubetSchemas", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "postgres_subset", + "description": "", + "label": "", + "type": "PostgresSourceSchemaSubset", + "longType": "PostgresSourceSchemaSubset", + "fullType": "mgmt.v1alpha1.PostgresSourceSchemaSubset", + "ismap": false, + "isoneof": true, + "oneofdecl": "schemas", + "defaultValue": "" + }, + { + "name": "mysql_subset", + "description": "", + "label": "", + "type": "MysqlSourceSchemaSubset", + "longType": "MysqlSourceSchemaSubset", + "fullType": "mgmt.v1alpha1.MysqlSourceSchemaSubset", + "ismap": false, + "isoneof": true, + "oneofdecl": "schemas", + "defaultValue": "" + } + ] + }, + { + "name": "JobStatusRecord", + "longName": "JobStatusRecord", + "fullName": "mgmt.v1alpha1.JobStatusRecord", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "status", + "description": "", + "label": "", + "type": "JobStatus", + "longType": "JobStatus", + "fullType": "mgmt.v1alpha1.JobStatus", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlDestinationConnectionOptions", + "longName": "MysqlDestinationConnectionOptions", + "fullName": "mgmt.v1alpha1.MysqlDestinationConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "truncate_table", + "description": "", + "label": "", + "type": "MysqlTruncateTableConfig", + "longType": "MysqlTruncateTableConfig", + "fullType": "mgmt.v1alpha1.MysqlTruncateTableConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "init_table_schema", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "on_conflict", + "description": "", + "label": "", + "type": "MysqlOnConflictConfig", + "longType": "MysqlOnConflictConfig", + "fullType": "mgmt.v1alpha1.MysqlOnConflictConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlOnConflictConfig", + "longName": "MysqlOnConflictConfig", + "fullName": "mgmt.v1alpha1.MysqlOnConflictConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "do_nothing", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlSourceConnectionOptions", + "longName": "MysqlSourceConnectionOptions", + "fullName": "mgmt.v1alpha1.MysqlSourceConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "halt_on_new_column_addition", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schemas", + "description": "", + "label": "repeated", + "type": "MysqlSourceSchemaOption", + "longType": "MysqlSourceSchemaOption", + "fullType": "mgmt.v1alpha1.MysqlSourceSchemaOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "subset_by_foreign_key_constraints", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlSourceSchemaOption", + "longName": "MysqlSourceSchemaOption", + "fullName": "mgmt.v1alpha1.MysqlSourceSchemaOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "tables", + "description": "", + "label": "repeated", + "type": "MysqlSourceTableOption", + "longType": "MysqlSourceTableOption", + "fullType": "mgmt.v1alpha1.MysqlSourceTableOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlSourceSchemaSubset", + "longName": "MysqlSourceSchemaSubset", + "fullName": "mgmt.v1alpha1.MysqlSourceSchemaSubset", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "mysql_schemas", + "description": "", + "label": "repeated", + "type": "MysqlSourceSchemaOption", + "longType": "MysqlSourceSchemaOption", + "fullType": "mgmt.v1alpha1.MysqlSourceSchemaOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlSourceTableOption", + "longName": "MysqlSourceTableOption", + "fullName": "mgmt.v1alpha1.MysqlSourceTableOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "where_clause", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_where_clause", + "defaultValue": "" + } + ] + }, + { + "name": "MysqlTruncateTableConfig", + "longName": "MysqlTruncateTableConfig", + "fullName": "mgmt.v1alpha1.MysqlTruncateTableConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "truncate_before_insert", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PauseJobRequest", + "longName": "PauseJobRequest", + "fullName": "mgmt.v1alpha1.PauseJobRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "pause", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "note", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_note", + "defaultValue": "" + } + ] + }, + { + "name": "PauseJobResponse", + "longName": "PauseJobResponse", + "fullName": "mgmt.v1alpha1.PauseJobResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PendingActivity", + "longName": "PendingActivity", + "fullName": "mgmt.v1alpha1.PendingActivity", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "status", + "description": "", + "label": "", + "type": "ActivityStatus", + "longType": "ActivityStatus", + "fullType": "mgmt.v1alpha1.ActivityStatus", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "activity_name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "last_failure", + "description": "", + "label": "optional", + "type": "ActivityFailure", + "longType": "ActivityFailure", + "fullType": "mgmt.v1alpha1.ActivityFailure", + "ismap": false, + "isoneof": true, + "oneofdecl": "_last_failure", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresDestinationConnectionOptions", + "longName": "PostgresDestinationConnectionOptions", + "fullName": "mgmt.v1alpha1.PostgresDestinationConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "truncate_table", + "description": "", + "label": "", + "type": "PostgresTruncateTableConfig", + "longType": "PostgresTruncateTableConfig", + "fullType": "mgmt.v1alpha1.PostgresTruncateTableConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "init_table_schema", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "on_conflict", + "description": "", + "label": "", + "type": "PostgresOnConflictConfig", + "longType": "PostgresOnConflictConfig", + "fullType": "mgmt.v1alpha1.PostgresOnConflictConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresOnConflictConfig", + "longName": "PostgresOnConflictConfig", + "fullName": "mgmt.v1alpha1.PostgresOnConflictConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "do_nothing", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresSourceConnectionOptions", + "longName": "PostgresSourceConnectionOptions", + "fullName": "mgmt.v1alpha1.PostgresSourceConnectionOptions", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "halt_on_new_column_addition", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schemas", + "description": "", + "label": "repeated", + "type": "PostgresSourceSchemaOption", + "longType": "PostgresSourceSchemaOption", + "fullType": "mgmt.v1alpha1.PostgresSourceSchemaOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "subset_by_foreign_key_constraints", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresSourceSchemaOption", + "longName": "PostgresSourceSchemaOption", + "fullName": "mgmt.v1alpha1.PostgresSourceSchemaOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "schema", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "tables", + "description": "", + "label": "repeated", + "type": "PostgresSourceTableOption", + "longType": "PostgresSourceTableOption", + "fullType": "mgmt.v1alpha1.PostgresSourceTableOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresSourceSchemaSubset", + "longName": "PostgresSourceSchemaSubset", + "fullName": "mgmt.v1alpha1.PostgresSourceSchemaSubset", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "postgres_schemas", + "description": "", + "label": "repeated", + "type": "PostgresSourceSchemaOption", + "longType": "PostgresSourceSchemaOption", + "fullType": "mgmt.v1alpha1.PostgresSourceSchemaOption", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresSourceTableOption", + "longName": "PostgresSourceTableOption", + "fullName": "mgmt.v1alpha1.PostgresSourceTableOption", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "table", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "where_clause", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_where_clause", + "defaultValue": "" + } + ] + }, + { + "name": "PostgresTruncateTableConfig", + "longName": "PostgresTruncateTableConfig", + "fullName": "mgmt.v1alpha1.PostgresTruncateTableConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "truncate_before_insert", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "cascade", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RetryPolicy", + "longName": "RetryPolicy", + "fullName": "mgmt.v1alpha1.RetryPolicy", + "description": "Defines the retry policy for an activity", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "maximum_attempts", + "description": "Maximum number of attempts. When exceeded the retries stop even if not expired yet.\nIf not set or set to 0, it means unlimited, and rely on activity ScheduleToCloseTimeout to stop.", + "label": "optional", + "type": "int32", + "longType": "int32", + "fullType": "int32", + "ismap": false, + "isoneof": true, + "oneofdecl": "_maximum_attempts", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobSourceSqlConnectionSubsetsRequest", + "longName": "SetJobSourceSqlConnectionSubsetsRequest", + "fullName": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "schemas", + "description": "", + "label": "", + "type": "JobSourceSqlSubetSchemas", + "longType": "JobSourceSqlSubetSchemas", + "fullType": "mgmt.v1alpha1.JobSourceSqlSubetSchemas", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "subset_by_foreign_key_constraints", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobSourceSqlConnectionSubsetsResponse", + "longName": "SetJobSourceSqlConnectionSubsetsResponse", + "fullName": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobSyncOptionsRequest", + "longName": "SetJobSyncOptionsRequest", + "fullName": "mgmt.v1alpha1.SetJobSyncOptionsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "The unique identifier of the job", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "sync_options", + "description": "The sync options object. The entire object must be provided and will fully overwrite the previous result", + "label": "", + "type": "ActivityOptions", + "longType": "ActivityOptions", + "fullType": "mgmt.v1alpha1.ActivityOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobSyncOptionsResponse", + "longName": "SetJobSyncOptionsResponse", + "fullName": "mgmt.v1alpha1.SetJobSyncOptionsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobWorkflowOptionsRequest", + "longName": "SetJobWorkflowOptionsRequest", + "fullName": "mgmt.v1alpha1.SetJobWorkflowOptionsRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "The unique identifier of the job", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "worfklow_options", + "description": "The workflow options object. The entire object must be provided and will fully overwrite the previous result", + "label": "", + "type": "WorkflowOptions", + "longType": "WorkflowOptions", + "fullType": "mgmt.v1alpha1.WorkflowOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetJobWorkflowOptionsResponse", + "longName": "SetJobWorkflowOptionsResponse", + "fullName": "mgmt.v1alpha1.SetJobWorkflowOptionsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TerminateJobRunRequest", + "longName": "TerminateJobRunRequest", + "fullName": "mgmt.v1alpha1.TerminateJobRunRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_run_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "TerminateJobRunResponse", + "longName": "TerminateJobRunResponse", + "fullName": "mgmt.v1alpha1.TerminateJobRunResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "UpdateJobDestinationConnectionRequest", + "longName": "UpdateJobDestinationConnectionRequest", + "fullName": "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "connection_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "options", + "description": "", + "label": "", + "type": "JobDestinationOptions", + "longType": "JobDestinationOptions", + "fullType": "mgmt.v1alpha1.JobDestinationOptions", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "destination_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateJobDestinationConnectionResponse", + "longName": "UpdateJobDestinationConnectionResponse", + "fullName": "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateJobScheduleRequest", + "longName": "UpdateJobScheduleRequest", + "fullName": "mgmt.v1alpha1.UpdateJobScheduleRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "cron_schedule", + "description": "", + "label": "optional", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "_cron_schedule", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateJobScheduleResponse", + "longName": "UpdateJobScheduleResponse", + "fullName": "mgmt.v1alpha1.UpdateJobScheduleResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateJobSourceConnectionRequest", + "longName": "UpdateJobSourceConnectionRequest", + "fullName": "mgmt.v1alpha1.UpdateJobSourceConnectionRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "source", + "description": "", + "label": "", + "type": "JobSource", + "longType": "JobSource", + "fullType": "mgmt.v1alpha1.JobSource", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "mappings", + "description": "", + "label": "repeated", + "type": "JobMapping", + "longType": "JobMapping", + "fullType": "mgmt.v1alpha1.JobMapping", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UpdateJobSourceConnectionResponse", + "longName": "UpdateJobSourceConnectionResponse", + "fullName": "mgmt.v1alpha1.UpdateJobSourceConnectionResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "job", + "description": "", + "label": "", + "type": "Job", + "longType": "Job", + "fullType": "mgmt.v1alpha1.Job", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "WorkflowOptions", + "longName": "WorkflowOptions", + "fullName": "mgmt.v1alpha1.WorkflowOptions", + "description": "Config that contains various timeouts that are configured in the underlying temporal workflow\nMore options will come in the future as needed", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "run_timeout", + "description": "The timeout for a single workflow run.\nMeasured in seconds", + "label": "optional", + "type": "int64", + "longType": "int64", + "fullType": "int64", + "ismap": false, + "isoneof": true, + "oneofdecl": "_run_timeout", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "JobService", + "longName": "JobService", + "fullName": "mgmt.v1alpha1.JobService", + "description": "", + "methods": [ + { + "name": "GetJobs", + "description": "", + "requestType": "GetJobsRequest", + "requestLongType": "GetJobsRequest", + "requestFullType": "mgmt.v1alpha1.GetJobsRequest", + "requestStreaming": false, + "responseType": "GetJobsResponse", + "responseLongType": "GetJobsResponse", + "responseFullType": "mgmt.v1alpha1.GetJobsResponse", + "responseStreaming": false + }, + { + "name": "GetJob", + "description": "", + "requestType": "GetJobRequest", + "requestLongType": "GetJobRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRequest", + "requestStreaming": false, + "responseType": "GetJobResponse", + "responseLongType": "GetJobResponse", + "responseFullType": "mgmt.v1alpha1.GetJobResponse", + "responseStreaming": false + }, + { + "name": "CreateJob", + "description": "", + "requestType": "CreateJobRequest", + "requestLongType": "CreateJobRequest", + "requestFullType": "mgmt.v1alpha1.CreateJobRequest", + "requestStreaming": false, + "responseType": "CreateJobResponse", + "responseLongType": "CreateJobResponse", + "responseFullType": "mgmt.v1alpha1.CreateJobResponse", + "responseStreaming": false + }, + { + "name": "DeleteJob", + "description": "", + "requestType": "DeleteJobRequest", + "requestLongType": "DeleteJobRequest", + "requestFullType": "mgmt.v1alpha1.DeleteJobRequest", + "requestStreaming": false, + "responseType": "DeleteJobResponse", + "responseLongType": "DeleteJobResponse", + "responseFullType": "mgmt.v1alpha1.DeleteJobResponse", + "responseStreaming": false + }, + { + "name": "IsJobNameAvailable", + "description": "", + "requestType": "IsJobNameAvailableRequest", + "requestLongType": "IsJobNameAvailableRequest", + "requestFullType": "mgmt.v1alpha1.IsJobNameAvailableRequest", + "requestStreaming": false, + "responseType": "IsJobNameAvailableResponse", + "responseLongType": "IsJobNameAvailableResponse", + "responseFullType": "mgmt.v1alpha1.IsJobNameAvailableResponse", + "responseStreaming": false + }, + { + "name": "UpdateJobSchedule", + "description": "", + "requestType": "UpdateJobScheduleRequest", + "requestLongType": "UpdateJobScheduleRequest", + "requestFullType": "mgmt.v1alpha1.UpdateJobScheduleRequest", + "requestStreaming": false, + "responseType": "UpdateJobScheduleResponse", + "responseLongType": "UpdateJobScheduleResponse", + "responseFullType": "mgmt.v1alpha1.UpdateJobScheduleResponse", + "responseStreaming": false + }, + { + "name": "UpdateJobSourceConnection", + "description": "", + "requestType": "UpdateJobSourceConnectionRequest", + "requestLongType": "UpdateJobSourceConnectionRequest", + "requestFullType": "mgmt.v1alpha1.UpdateJobSourceConnectionRequest", + "requestStreaming": false, + "responseType": "UpdateJobSourceConnectionResponse", + "responseLongType": "UpdateJobSourceConnectionResponse", + "responseFullType": "mgmt.v1alpha1.UpdateJobSourceConnectionResponse", + "responseStreaming": false + }, + { + "name": "SetJobSourceSqlConnectionSubsets", + "description": "", + "requestType": "SetJobSourceSqlConnectionSubsetsRequest", + "requestLongType": "SetJobSourceSqlConnectionSubsetsRequest", + "requestFullType": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest", + "requestStreaming": false, + "responseType": "SetJobSourceSqlConnectionSubsetsResponse", + "responseLongType": "SetJobSourceSqlConnectionSubsetsResponse", + "responseFullType": "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse", + "responseStreaming": false + }, + { + "name": "UpdateJobDestinationConnection", + "description": "", + "requestType": "UpdateJobDestinationConnectionRequest", + "requestLongType": "UpdateJobDestinationConnectionRequest", + "requestFullType": "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest", + "requestStreaming": false, + "responseType": "UpdateJobDestinationConnectionResponse", + "responseLongType": "UpdateJobDestinationConnectionResponse", + "responseFullType": "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse", + "responseStreaming": false + }, + { + "name": "DeleteJobDestinationConnection", + "description": "", + "requestType": "DeleteJobDestinationConnectionRequest", + "requestLongType": "DeleteJobDestinationConnectionRequest", + "requestFullType": "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest", + "requestStreaming": false, + "responseType": "DeleteJobDestinationConnectionResponse", + "responseLongType": "DeleteJobDestinationConnectionResponse", + "responseFullType": "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse", + "responseStreaming": false + }, + { + "name": "CreateJobDestinationConnections", + "description": "", + "requestType": "CreateJobDestinationConnectionsRequest", + "requestLongType": "CreateJobDestinationConnectionsRequest", + "requestFullType": "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest", + "requestStreaming": false, + "responseType": "CreateJobDestinationConnectionsResponse", + "responseLongType": "CreateJobDestinationConnectionsResponse", + "responseFullType": "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse", + "responseStreaming": false + }, + { + "name": "PauseJob", + "description": "", + "requestType": "PauseJobRequest", + "requestLongType": "PauseJobRequest", + "requestFullType": "mgmt.v1alpha1.PauseJobRequest", + "requestStreaming": false, + "responseType": "PauseJobResponse", + "responseLongType": "PauseJobResponse", + "responseFullType": "mgmt.v1alpha1.PauseJobResponse", + "responseStreaming": false + }, + { + "name": "GetJobRecentRuns", + "description": "Returns a list of recently invoked job runs baseds on the Temporal cron scheduler. This will return a list of job runs that include archived runs", + "requestType": "GetJobRecentRunsRequest", + "requestLongType": "GetJobRecentRunsRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRecentRunsRequest", + "requestStreaming": false, + "responseType": "GetJobRecentRunsResponse", + "responseLongType": "GetJobRecentRunsResponse", + "responseFullType": "mgmt.v1alpha1.GetJobRecentRunsResponse", + "responseStreaming": false + }, + { + "name": "GetJobNextRuns", + "description": "Returns a list of runs that are scheduled for execution based on the Temporal cron scheduler.", + "requestType": "GetJobNextRunsRequest", + "requestLongType": "GetJobNextRunsRequest", + "requestFullType": "mgmt.v1alpha1.GetJobNextRunsRequest", + "requestStreaming": false, + "responseType": "GetJobNextRunsResponse", + "responseLongType": "GetJobNextRunsResponse", + "responseFullType": "mgmt.v1alpha1.GetJobNextRunsResponse", + "responseStreaming": false + }, + { + "name": "GetJobStatus", + "description": "", + "requestType": "GetJobStatusRequest", + "requestLongType": "GetJobStatusRequest", + "requestFullType": "mgmt.v1alpha1.GetJobStatusRequest", + "requestStreaming": false, + "responseType": "GetJobStatusResponse", + "responseLongType": "GetJobStatusResponse", + "responseFullType": "mgmt.v1alpha1.GetJobStatusResponse", + "responseStreaming": false + }, + { + "name": "GetJobStatuses", + "description": "", + "requestType": "GetJobStatusesRequest", + "requestLongType": "GetJobStatusesRequest", + "requestFullType": "mgmt.v1alpha1.GetJobStatusesRequest", + "requestStreaming": false, + "responseType": "GetJobStatusesResponse", + "responseLongType": "GetJobStatusesResponse", + "responseFullType": "mgmt.v1alpha1.GetJobStatusesResponse", + "responseStreaming": false + }, + { + "name": "GetJobRuns", + "description": "Returns a list of job runs by either account or job", + "requestType": "GetJobRunsRequest", + "requestLongType": "GetJobRunsRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRunsRequest", + "requestStreaming": false, + "responseType": "GetJobRunsResponse", + "responseLongType": "GetJobRunsResponse", + "responseFullType": "mgmt.v1alpha1.GetJobRunsResponse", + "responseStreaming": false + }, + { + "name": "GetJobRunEvents", + "description": "", + "requestType": "GetJobRunEventsRequest", + "requestLongType": "GetJobRunEventsRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRunEventsRequest", + "requestStreaming": false, + "responseType": "GetJobRunEventsResponse", + "responseLongType": "GetJobRunEventsResponse", + "responseFullType": "mgmt.v1alpha1.GetJobRunEventsResponse", + "responseStreaming": false + }, + { + "name": "GetJobRun", + "description": "Returns a specific job run, along with any of its pending activities", + "requestType": "GetJobRunRequest", + "requestLongType": "GetJobRunRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRunRequest", + "requestStreaming": false, + "responseType": "GetJobRunResponse", + "responseLongType": "GetJobRunResponse", + "responseFullType": "mgmt.v1alpha1.GetJobRunResponse", + "responseStreaming": false + }, + { + "name": "DeleteJobRun", + "description": "", + "requestType": "DeleteJobRunRequest", + "requestLongType": "DeleteJobRunRequest", + "requestFullType": "mgmt.v1alpha1.DeleteJobRunRequest", + "requestStreaming": false, + "responseType": "DeleteJobRunResponse", + "responseLongType": "DeleteJobRunResponse", + "responseFullType": "mgmt.v1alpha1.DeleteJobRunResponse", + "responseStreaming": false + }, + { + "name": "CreateJobRun", + "description": "", + "requestType": "CreateJobRunRequest", + "requestLongType": "CreateJobRunRequest", + "requestFullType": "mgmt.v1alpha1.CreateJobRunRequest", + "requestStreaming": false, + "responseType": "CreateJobRunResponse", + "responseLongType": "CreateJobRunResponse", + "responseFullType": "mgmt.v1alpha1.CreateJobRunResponse", + "responseStreaming": false + }, + { + "name": "CancelJobRun", + "description": "", + "requestType": "CancelJobRunRequest", + "requestLongType": "CancelJobRunRequest", + "requestFullType": "mgmt.v1alpha1.CancelJobRunRequest", + "requestStreaming": false, + "responseType": "CancelJobRunResponse", + "responseLongType": "CancelJobRunResponse", + "responseFullType": "mgmt.v1alpha1.CancelJobRunResponse", + "responseStreaming": false + }, + { + "name": "TerminateJobRun", + "description": "", + "requestType": "TerminateJobRunRequest", + "requestLongType": "TerminateJobRunRequest", + "requestFullType": "mgmt.v1alpha1.TerminateJobRunRequest", + "requestStreaming": false, + "responseType": "TerminateJobRunResponse", + "responseLongType": "TerminateJobRunResponse", + "responseFullType": "mgmt.v1alpha1.TerminateJobRunResponse", + "responseStreaming": false + }, + { + "name": "GetJobRunLogsStream", + "description": "Returns a stream of logs from the worker nodes that pertain to a specific job run", + "requestType": "GetJobRunLogsStreamRequest", + "requestLongType": "GetJobRunLogsStreamRequest", + "requestFullType": "mgmt.v1alpha1.GetJobRunLogsStreamRequest", + "requestStreaming": false, + "responseType": "GetJobRunLogsStreamResponse", + "responseLongType": "GetJobRunLogsStreamResponse", + "responseFullType": "mgmt.v1alpha1.GetJobRunLogsStreamResponse", + "responseStreaming": true + }, + { + "name": "SetJobWorkflowOptions", + "description": "Set any job workflow options. Must provide entire object as is it will fully override the previous configuration", + "requestType": "SetJobWorkflowOptionsRequest", + "requestLongType": "SetJobWorkflowOptionsRequest", + "requestFullType": "mgmt.v1alpha1.SetJobWorkflowOptionsRequest", + "requestStreaming": false, + "responseType": "SetJobWorkflowOptionsResponse", + "responseLongType": "SetJobWorkflowOptionsResponse", + "responseFullType": "mgmt.v1alpha1.SetJobWorkflowOptionsResponse", + "responseStreaming": false + }, + { + "name": "SetJobSyncOptions", + "description": "Set the job sync options. Must provide entire object as it will fully override the previous configuration", + "requestType": "SetJobSyncOptionsRequest", + "requestLongType": "SetJobSyncOptionsRequest", + "requestFullType": "mgmt.v1alpha1.SetJobSyncOptionsRequest", + "requestStreaming": false, + "responseType": "SetJobSyncOptionsResponse", + "responseLongType": "SetJobSyncOptionsResponse", + "responseFullType": "mgmt.v1alpha1.SetJobSyncOptionsResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/metrics.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": true, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [ + { + "name": "RangedMetricName", + "longName": "RangedMetricName", + "fullName": "mgmt.v1alpha1.RangedMetricName", + "description": "", + "values": [ + { + "name": "RANGED_METRIC_NAME_UNSPECIFIED", + "number": "0", + "description": "If unspecified, an error will be thrown" + }, + { + "name": "RANGED_METRIC_NAME_INPUT_RECEIVED", + "number": "1", + "description": "The input_received metric" + } + ] + } + ], + "extensions": [], + "messages": [ + { + "name": "Date", + "longName": "Date", + "fullName": "mgmt.v1alpha1.Date", + "description": "Represents a whole or partial calendar date, such as a birthday. The time of\nday and time zone are either specified elsewhere or are insignificant. The\ndate is relative to the Gregorian Calendar. This can represent one of the\nfollowing:\n\n* A full date, with non-zero year, month, and day values\n* A month and day value, with a zero year, such as an anniversary\n* A year on its own, with zero month and day values\n* A year and month value, with a zero day, such as a credit card expiration\ndate\n\nRelated types are [google.type.TimeOfDay][google.type.TimeOfDay] and\n`google.protobuf.Timestamp`.", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "year", + "description": "Year of the date. Must be from 1 to 9999, or 0 to specify a date without\na year.", + "label": "", + "type": "uint32", + "longType": "uint32", + "fullType": "uint32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "month", + "description": "Month of a year. Must be from 1 to 12, or 0 to specify a year without a\nmonth and day.", + "label": "", + "type": "uint32", + "longType": "uint32", + "fullType": "uint32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "day", + "description": "Day of a month. Must be from 1 to 31 and valid for the year and month, or 0\nto specify a year by itself or a year and month where the day isn't\nsignificant.", + "label": "", + "type": "uint32", + "longType": "uint32", + "fullType": "uint32", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "DayResult", + "longName": "DayResult", + "fullName": "mgmt.v1alpha1.DayResult", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "date", + "description": "", + "label": "", + "type": "Date", + "longType": "Date", + "fullType": "mgmt.v1alpha1.Date", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "count", + "description": "", + "label": "", + "type": "uint64", + "longType": "uint64", + "fullType": "uint64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetDailyMetricCountRequest", + "longName": "GetDailyMetricCountRequest", + "fullName": "mgmt.v1alpha1.GetDailyMetricCountRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "start", + "description": "The start day", + "label": "", + "type": "Date", + "longType": "Date", + "fullType": "mgmt.v1alpha1.Date", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "end", + "description": "The end day", + "label": "", + "type": "Date", + "longType": "Date", + "fullType": "mgmt.v1alpha1.Date", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "metric", + "description": "The metric to return", + "label": "", + "type": "RangedMetricName", + "longType": "RangedMetricName", + "fullType": "mgmt.v1alpha1.RangedMetricName", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "The account identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + }, + { + "name": "job_id", + "description": "The job identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + }, + { + "name": "run_id", + "description": "The run identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + } + ] + }, + { + "name": "GetDailyMetricCountResponse", + "longName": "GetDailyMetricCountResponse", + "fullName": "mgmt.v1alpha1.GetDailyMetricCountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "results", + "description": "", + "label": "repeated", + "type": "DayResult", + "longType": "DayResult", + "fullType": "mgmt.v1alpha1.DayResult", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetMetricCountRequest", + "longName": "GetMetricCountRequest", + "fullName": "mgmt.v1alpha1.GetMetricCountRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": true, + "extensions": [], + "fields": [ + { + "name": "start", + "description": "The start time", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "end", + "description": "The end time", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "metric", + "description": "The metric to return", + "label": "", + "type": "RangedMetricName", + "longType": "RangedMetricName", + "fullType": "mgmt.v1alpha1.RangedMetricName", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "The account identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + }, + { + "name": "job_id", + "description": "The job identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + }, + { + "name": "run_id", + "description": "The run identifier that will be used to filter by", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": true, + "oneofdecl": "identifier", + "defaultValue": "" + } + ] + }, + { + "name": "GetMetricCountResponse", + "longName": "GetMetricCountResponse", + "fullName": "mgmt.v1alpha1.GetMetricCountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "count", + "description": "The summed up count of the metric based on the input query and timerange specified", + "label": "", + "type": "uint64", + "longType": "uint64", + "fullType": "uint64", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "MetricsService", + "longName": "MetricsService", + "fullName": "mgmt.v1alpha1.MetricsService", + "description": "", + "methods": [ + { + "name": "GetDailyMetricCount", + "description": "Retrieve a timed range of records", + "requestType": "GetDailyMetricCountRequest", + "requestLongType": "GetDailyMetricCountRequest", + "requestFullType": "mgmt.v1alpha1.GetDailyMetricCountRequest", + "requestStreaming": false, + "responseType": "GetDailyMetricCountResponse", + "responseLongType": "GetDailyMetricCountResponse", + "responseFullType": "mgmt.v1alpha1.GetDailyMetricCountResponse", + "responseStreaming": false + }, + { + "name": "GetMetricCount", + "description": "For the given metric and time range, returns the total count found", + "requestType": "GetMetricCountRequest", + "requestLongType": "GetMetricCountRequest", + "requestFullType": "mgmt.v1alpha1.GetMetricCountRequest", + "requestStreaming": false, + "responseType": "GetMetricCountResponse", + "responseLongType": "GetMetricCountResponse", + "responseFullType": "mgmt.v1alpha1.GetMetricCountResponse", + "responseStreaming": false + } + ] + } + ] + }, + { + "name": "mgmt/v1alpha1/user_account.proto", + "description": "", + "package": "mgmt.v1alpha1", + "hasEnums": true, + "hasExtensions": false, + "hasMessages": true, + "hasServices": true, + "enums": [ + { + "name": "UserAccountType", + "longName": "UserAccountType", + "fullName": "mgmt.v1alpha1.UserAccountType", + "description": "", + "values": [ + { + "name": "USER_ACCOUNT_TYPE_UNSPECIFIED", + "number": "0", + "description": "" + }, + { + "name": "USER_ACCOUNT_TYPE_PERSONAL", + "number": "1", + "description": "" + }, + { + "name": "USER_ACCOUNT_TYPE_TEAM", + "number": "2", + "description": "" + }, + { + "name": "USER_ACCOUNT_TYPE_ENTERPRISE", + "number": "3", + "description": "" + } + ] + } + ], + "extensions": [], + "messages": [ + { + "name": "AcceptTeamAccountInviteRequest", + "longName": "AcceptTeamAccountInviteRequest", + "fullName": "mgmt.v1alpha1.AcceptTeamAccountInviteRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "token", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AcceptTeamAccountInviteResponse", + "longName": "AcceptTeamAccountInviteResponse", + "fullName": "mgmt.v1alpha1.AcceptTeamAccountInviteResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account", + "description": "", + "label": "", + "type": "UserAccount", + "longType": "UserAccount", + "fullType": "mgmt.v1alpha1.UserAccount", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AccountInvite", + "longName": "AccountInvite", + "fullName": "mgmt.v1alpha1.AccountInvite", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "sender_user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "email", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "token", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "accepted", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "created_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "updated_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "expires_at", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AccountOnboardingConfig", + "longName": "AccountOnboardingConfig", + "fullName": "mgmt.v1alpha1.AccountOnboardingConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "has_created_source_connection", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "has_created_destination_connection", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "has_created_job", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "has_invited_members", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AccountTemporalConfig", + "longName": "AccountTemporalConfig", + "fullName": "mgmt.v1alpha1.AccountTemporalConfig", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "url", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "namespace", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "sync_job_queue_name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "AccountUser", + "longName": "AccountUser", + "fullName": "mgmt.v1alpha1.AccountUser", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "image", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "email", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "ConvertPersonalToTeamAccountRequest", + "longName": "ConvertPersonalToTeamAccountRequest", + "fullName": "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "ConvertPersonalToTeamAccountResponse", + "longName": "ConvertPersonalToTeamAccountResponse", + "fullName": "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "CreateTeamAccountRequest", + "longName": "CreateTeamAccountRequest", + "fullName": "mgmt.v1alpha1.CreateTeamAccountRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "CreateTeamAccountResponse", + "longName": "CreateTeamAccountResponse", + "fullName": "mgmt.v1alpha1.CreateTeamAccountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountOnboardingConfigRequest", + "longName": "GetAccountOnboardingConfigRequest", + "fullName": "mgmt.v1alpha1.GetAccountOnboardingConfigRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountOnboardingConfigResponse", + "longName": "GetAccountOnboardingConfigResponse", + "fullName": "mgmt.v1alpha1.GetAccountOnboardingConfigResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "config", + "description": "", + "label": "", + "type": "AccountOnboardingConfig", + "longType": "AccountOnboardingConfig", + "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountTemporalConfigRequest", + "longName": "GetAccountTemporalConfigRequest", + "fullName": "mgmt.v1alpha1.GetAccountTemporalConfigRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetAccountTemporalConfigResponse", + "longName": "GetAccountTemporalConfigResponse", + "fullName": "mgmt.v1alpha1.GetAccountTemporalConfigResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "config", + "description": "", + "label": "", + "type": "AccountTemporalConfig", + "longType": "AccountTemporalConfig", + "fullType": "mgmt.v1alpha1.AccountTemporalConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetSystemInformationRequest", + "longName": "GetSystemInformationRequest", + "fullName": "mgmt.v1alpha1.GetSystemInformationRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetSystemInformationResponse", + "longName": "GetSystemInformationResponse", + "fullName": "mgmt.v1alpha1.GetSystemInformationResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "version", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "commit", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "compiler", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "platform", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "build_date", + "description": "", + "label": "", + "type": "Timestamp", + "longType": "google.protobuf.Timestamp", + "fullType": "google.protobuf.Timestamp", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetTeamAccountInvitesRequest", + "longName": "GetTeamAccountInvitesRequest", + "fullName": "mgmt.v1alpha1.GetTeamAccountInvitesRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetTeamAccountInvitesResponse", + "longName": "GetTeamAccountInvitesResponse", + "fullName": "mgmt.v1alpha1.GetTeamAccountInvitesResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "invites", + "description": "", + "label": "repeated", + "type": "AccountInvite", + "longType": "AccountInvite", + "fullType": "mgmt.v1alpha1.AccountInvite", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetTeamAccountMembersRequest", + "longName": "GetTeamAccountMembersRequest", + "fullName": "mgmt.v1alpha1.GetTeamAccountMembersRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetTeamAccountMembersResponse", + "longName": "GetTeamAccountMembersResponse", + "fullName": "mgmt.v1alpha1.GetTeamAccountMembersResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "users", + "description": "", + "label": "repeated", + "type": "AccountUser", + "longType": "AccountUser", + "fullType": "mgmt.v1alpha1.AccountUser", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserAccountsRequest", + "longName": "GetUserAccountsRequest", + "fullName": "mgmt.v1alpha1.GetUserAccountsRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetUserAccountsResponse", + "longName": "GetUserAccountsResponse", + "fullName": "mgmt.v1alpha1.GetUserAccountsResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "accounts", + "description": "", + "label": "repeated", + "type": "UserAccount", + "longType": "UserAccount", + "fullType": "mgmt.v1alpha1.UserAccount", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "GetUserRequest", + "longName": "GetUserRequest", + "fullName": "mgmt.v1alpha1.GetUserRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "GetUserResponse", + "longName": "GetUserResponse", + "fullName": "mgmt.v1alpha1.GetUserResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "InviteUserToTeamAccountRequest", + "longName": "InviteUserToTeamAccountRequest", + "fullName": "mgmt.v1alpha1.InviteUserToTeamAccountRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "email", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "InviteUserToTeamAccountResponse", + "longName": "InviteUserToTeamAccountResponse", + "fullName": "mgmt.v1alpha1.InviteUserToTeamAccountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "invite", + "description": "", + "label": "", + "type": "AccountInvite", + "longType": "AccountInvite", + "fullType": "mgmt.v1alpha1.AccountInvite", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsUserInAccountRequest", + "longName": "IsUserInAccountRequest", + "fullName": "mgmt.v1alpha1.IsUserInAccountRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "IsUserInAccountResponse", + "longName": "IsUserInAccountResponse", + "fullName": "mgmt.v1alpha1.IsUserInAccountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "ok", + "description": "", + "label": "", + "type": "bool", + "longType": "bool", + "fullType": "bool", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RemoveTeamAccountInviteRequest", + "longName": "RemoveTeamAccountInviteRequest", + "fullName": "mgmt.v1alpha1.RemoveTeamAccountInviteRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RemoveTeamAccountInviteResponse", + "longName": "RemoveTeamAccountInviteResponse", + "fullName": "mgmt.v1alpha1.RemoveTeamAccountInviteResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "RemoveTeamAccountMemberRequest", + "longName": "RemoveTeamAccountMemberRequest", + "fullName": "mgmt.v1alpha1.RemoveTeamAccountMemberRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "RemoveTeamAccountMemberResponse", + "longName": "RemoveTeamAccountMemberResponse", + "fullName": "mgmt.v1alpha1.RemoveTeamAccountMemberResponse", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "SetAccountOnboardingConfigRequest", + "longName": "SetAccountOnboardingConfigRequest", + "fullName": "mgmt.v1alpha1.SetAccountOnboardingConfigRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "config", + "description": "", + "label": "", + "type": "AccountOnboardingConfig", + "longType": "AccountOnboardingConfig", + "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetAccountOnboardingConfigResponse", + "longName": "SetAccountOnboardingConfigResponse", + "fullName": "mgmt.v1alpha1.SetAccountOnboardingConfigResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "config", + "description": "", + "label": "", + "type": "AccountOnboardingConfig", + "longType": "AccountOnboardingConfig", + "fullType": "mgmt.v1alpha1.AccountOnboardingConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetAccountTemporalConfigRequest", + "longName": "SetAccountTemporalConfigRequest", + "fullName": "mgmt.v1alpha1.SetAccountTemporalConfigRequest", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "config", + "description": "", + "label": "", + "type": "AccountTemporalConfig", + "longType": "AccountTemporalConfig", + "fullType": "mgmt.v1alpha1.AccountTemporalConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetAccountTemporalConfigResponse", + "longName": "SetAccountTemporalConfigResponse", + "fullName": "mgmt.v1alpha1.SetAccountTemporalConfigResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "config", + "description": "", + "label": "", + "type": "AccountTemporalConfig", + "longType": "AccountTemporalConfig", + "fullType": "mgmt.v1alpha1.AccountTemporalConfig", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetPersonalAccountRequest", + "longName": "SetPersonalAccountRequest", + "fullName": "mgmt.v1alpha1.SetPersonalAccountRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "SetPersonalAccountResponse", + "longName": "SetPersonalAccountResponse", + "fullName": "mgmt.v1alpha1.SetPersonalAccountResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "account_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "SetUserRequest", + "longName": "SetUserRequest", + "fullName": "mgmt.v1alpha1.SetUserRequest", + "description": "", + "hasExtensions": false, + "hasFields": false, + "hasOneofs": false, + "extensions": [], + "fields": [] + }, + { + "name": "SetUserResponse", + "longName": "SetUserResponse", + "fullName": "mgmt.v1alpha1.SetUserResponse", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "user_id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + }, + { + "name": "UserAccount", + "longName": "UserAccount", + "fullName": "mgmt.v1alpha1.UserAccount", + "description": "", + "hasExtensions": false, + "hasFields": true, + "hasOneofs": false, + "extensions": [], + "fields": [ + { + "name": "id", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "name", + "description": "", + "label": "", + "type": "string", + "longType": "string", + "fullType": "string", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + }, + { + "name": "type", + "description": "", + "label": "", + "type": "UserAccountType", + "longType": "UserAccountType", + "fullType": "mgmt.v1alpha1.UserAccountType", + "ismap": false, + "isoneof": false, + "oneofdecl": "", + "defaultValue": "" + } + ] + } + ], + "services": [ + { + "name": "UserAccountService", + "longName": "UserAccountService", + "fullName": "mgmt.v1alpha1.UserAccountService", + "description": "", + "methods": [ + { + "name": "GetUser", + "description": "", + "requestType": "GetUserRequest", + "requestLongType": "GetUserRequest", + "requestFullType": "mgmt.v1alpha1.GetUserRequest", + "requestStreaming": false, + "responseType": "GetUserResponse", + "responseLongType": "GetUserResponse", + "responseFullType": "mgmt.v1alpha1.GetUserResponse", + "responseStreaming": false + }, + { + "name": "SetUser", + "description": "", + "requestType": "SetUserRequest", + "requestLongType": "SetUserRequest", + "requestFullType": "mgmt.v1alpha1.SetUserRequest", + "requestStreaming": false, + "responseType": "SetUserResponse", + "responseLongType": "SetUserResponse", + "responseFullType": "mgmt.v1alpha1.SetUserResponse", + "responseStreaming": false + }, + { + "name": "GetUserAccounts", + "description": "", + "requestType": "GetUserAccountsRequest", + "requestLongType": "GetUserAccountsRequest", + "requestFullType": "mgmt.v1alpha1.GetUserAccountsRequest", + "requestStreaming": false, + "responseType": "GetUserAccountsResponse", + "responseLongType": "GetUserAccountsResponse", + "responseFullType": "mgmt.v1alpha1.GetUserAccountsResponse", + "responseStreaming": false + }, + { + "name": "SetPersonalAccount", + "description": "", + "requestType": "SetPersonalAccountRequest", + "requestLongType": "SetPersonalAccountRequest", + "requestFullType": "mgmt.v1alpha1.SetPersonalAccountRequest", + "requestStreaming": false, + "responseType": "SetPersonalAccountResponse", + "responseLongType": "SetPersonalAccountResponse", + "responseFullType": "mgmt.v1alpha1.SetPersonalAccountResponse", + "responseStreaming": false + }, + { + "name": "ConvertPersonalToTeamAccount", + "description": "", + "requestType": "ConvertPersonalToTeamAccountRequest", + "requestLongType": "ConvertPersonalToTeamAccountRequest", + "requestFullType": "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest", + "requestStreaming": false, + "responseType": "ConvertPersonalToTeamAccountResponse", + "responseLongType": "ConvertPersonalToTeamAccountResponse", + "responseFullType": "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse", + "responseStreaming": false + }, + { + "name": "CreateTeamAccount", + "description": "", + "requestType": "CreateTeamAccountRequest", + "requestLongType": "CreateTeamAccountRequest", + "requestFullType": "mgmt.v1alpha1.CreateTeamAccountRequest", + "requestStreaming": false, + "responseType": "CreateTeamAccountResponse", + "responseLongType": "CreateTeamAccountResponse", + "responseFullType": "mgmt.v1alpha1.CreateTeamAccountResponse", + "responseStreaming": false + }, + { + "name": "IsUserInAccount", + "description": "", + "requestType": "IsUserInAccountRequest", + "requestLongType": "IsUserInAccountRequest", + "requestFullType": "mgmt.v1alpha1.IsUserInAccountRequest", + "requestStreaming": false, + "responseType": "IsUserInAccountResponse", + "responseLongType": "IsUserInAccountResponse", + "responseFullType": "mgmt.v1alpha1.IsUserInAccountResponse", + "responseStreaming": false + }, + { + "name": "GetAccountTemporalConfig", + "description": "", + "requestType": "GetAccountTemporalConfigRequest", + "requestLongType": "GetAccountTemporalConfigRequest", + "requestFullType": "mgmt.v1alpha1.GetAccountTemporalConfigRequest", + "requestStreaming": false, + "responseType": "GetAccountTemporalConfigResponse", + "responseLongType": "GetAccountTemporalConfigResponse", + "responseFullType": "mgmt.v1alpha1.GetAccountTemporalConfigResponse", + "responseStreaming": false + }, + { + "name": "SetAccountTemporalConfig", + "description": "", + "requestType": "SetAccountTemporalConfigRequest", + "requestLongType": "SetAccountTemporalConfigRequest", + "requestFullType": "mgmt.v1alpha1.SetAccountTemporalConfigRequest", + "requestStreaming": false, + "responseType": "SetAccountTemporalConfigResponse", + "responseLongType": "SetAccountTemporalConfigResponse", + "responseFullType": "mgmt.v1alpha1.SetAccountTemporalConfigResponse", + "responseStreaming": false + }, + { + "name": "GetTeamAccountMembers", + "description": "", + "requestType": "GetTeamAccountMembersRequest", + "requestLongType": "GetTeamAccountMembersRequest", + "requestFullType": "mgmt.v1alpha1.GetTeamAccountMembersRequest", + "requestStreaming": false, + "responseType": "GetTeamAccountMembersResponse", + "responseLongType": "GetTeamAccountMembersResponse", + "responseFullType": "mgmt.v1alpha1.GetTeamAccountMembersResponse", + "responseStreaming": false + }, + { + "name": "RemoveTeamAccountMember", + "description": "", + "requestType": "RemoveTeamAccountMemberRequest", + "requestLongType": "RemoveTeamAccountMemberRequest", + "requestFullType": "mgmt.v1alpha1.RemoveTeamAccountMemberRequest", + "requestStreaming": false, + "responseType": "RemoveTeamAccountMemberResponse", + "responseLongType": "RemoveTeamAccountMemberResponse", + "responseFullType": "mgmt.v1alpha1.RemoveTeamAccountMemberResponse", + "responseStreaming": false + }, + { + "name": "InviteUserToTeamAccount", + "description": "", + "requestType": "InviteUserToTeamAccountRequest", + "requestLongType": "InviteUserToTeamAccountRequest", + "requestFullType": "mgmt.v1alpha1.InviteUserToTeamAccountRequest", + "requestStreaming": false, + "responseType": "InviteUserToTeamAccountResponse", + "responseLongType": "InviteUserToTeamAccountResponse", + "responseFullType": "mgmt.v1alpha1.InviteUserToTeamAccountResponse", + "responseStreaming": false + }, + { + "name": "GetTeamAccountInvites", + "description": "", + "requestType": "GetTeamAccountInvitesRequest", + "requestLongType": "GetTeamAccountInvitesRequest", + "requestFullType": "mgmt.v1alpha1.GetTeamAccountInvitesRequest", + "requestStreaming": false, + "responseType": "GetTeamAccountInvitesResponse", + "responseLongType": "GetTeamAccountInvitesResponse", + "responseFullType": "mgmt.v1alpha1.GetTeamAccountInvitesResponse", + "responseStreaming": false + }, + { + "name": "RemoveTeamAccountInvite", + "description": "", + "requestType": "RemoveTeamAccountInviteRequest", + "requestLongType": "RemoveTeamAccountInviteRequest", + "requestFullType": "mgmt.v1alpha1.RemoveTeamAccountInviteRequest", + "requestStreaming": false, + "responseType": "RemoveTeamAccountInviteResponse", + "responseLongType": "RemoveTeamAccountInviteResponse", + "responseFullType": "mgmt.v1alpha1.RemoveTeamAccountInviteResponse", + "responseStreaming": false + }, + { + "name": "AcceptTeamAccountInvite", + "description": "", + "requestType": "AcceptTeamAccountInviteRequest", + "requestLongType": "AcceptTeamAccountInviteRequest", + "requestFullType": "mgmt.v1alpha1.AcceptTeamAccountInviteRequest", + "requestStreaming": false, + "responseType": "AcceptTeamAccountInviteResponse", + "responseLongType": "AcceptTeamAccountInviteResponse", + "responseFullType": "mgmt.v1alpha1.AcceptTeamAccountInviteResponse", + "responseStreaming": false + }, + { + "name": "GetSystemInformation", + "description": "", + "requestType": "GetSystemInformationRequest", + "requestLongType": "GetSystemInformationRequest", + "requestFullType": "mgmt.v1alpha1.GetSystemInformationRequest", + "requestStreaming": false, + "responseType": "GetSystemInformationResponse", + "responseLongType": "GetSystemInformationResponse", + "responseFullType": "mgmt.v1alpha1.GetSystemInformationResponse", + "responseStreaming": false + }, + { + "name": "GetAccountOnboardingConfig", + "description": "", + "requestType": "GetAccountOnboardingConfigRequest", + "requestLongType": "GetAccountOnboardingConfigRequest", + "requestFullType": "mgmt.v1alpha1.GetAccountOnboardingConfigRequest", + "requestStreaming": false, + "responseType": "GetAccountOnboardingConfigResponse", + "responseLongType": "GetAccountOnboardingConfigResponse", + "responseFullType": "mgmt.v1alpha1.GetAccountOnboardingConfigResponse", + "responseStreaming": false + }, + { + "name": "SetAccountOnboardingConfig", + "description": "", + "requestType": "SetAccountOnboardingConfigRequest", + "requestLongType": "SetAccountOnboardingConfigRequest", + "requestFullType": "mgmt.v1alpha1.SetAccountOnboardingConfigRequest", + "requestStreaming": false, + "responseType": "SetAccountOnboardingConfigResponse", + "responseLongType": "SetAccountOnboardingConfigResponse", + "responseFullType": "mgmt.v1alpha1.SetAccountOnboardingConfigResponse", + "responseStreaming": false + } + ] + } + ] + } + ], + "scalarValueTypes": [ + { + "protoType": "double", + "notes": "", + "cppType": "double", + "csType": "double", + "goType": "float64", + "javaType": "double", + "phpType": "float", + "pythonType": "float", + "rubyType": "Float" + }, + { + "protoType": "float", + "notes": "", + "cppType": "float", + "csType": "float", + "goType": "float32", + "javaType": "float", + "phpType": "float", + "pythonType": "float", + "rubyType": "Float" + }, + { + "protoType": "int32", + "notes": "Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint32 instead.", + "cppType": "int32", + "csType": "int", + "goType": "int32", + "javaType": "int", + "phpType": "integer", + "pythonType": "int", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "int64", + "notes": "Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint64 instead.", + "cppType": "int64", + "csType": "long", + "goType": "int64", + "javaType": "long", + "phpType": "integer/string", + "pythonType": "int/long", + "rubyType": "Bignum" + }, + { + "protoType": "uint32", + "notes": "Uses variable-length encoding.", + "cppType": "uint32", + "csType": "uint", + "goType": "uint32", + "javaType": "int", + "phpType": "integer", + "pythonType": "int/long", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "uint64", + "notes": "Uses variable-length encoding.", + "cppType": "uint64", + "csType": "ulong", + "goType": "uint64", + "javaType": "long", + "phpType": "integer/string", + "pythonType": "int/long", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "sint32", + "notes": "Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int32s.", + "cppType": "int32", + "csType": "int", + "goType": "int32", + "javaType": "int", + "phpType": "integer", + "pythonType": "int", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "sint64", + "notes": "Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int64s.", + "cppType": "int64", + "csType": "long", + "goType": "int64", + "javaType": "long", + "phpType": "integer/string", + "pythonType": "int/long", + "rubyType": "Bignum" + }, + { + "protoType": "fixed32", + "notes": "Always four bytes. More efficient than uint32 if values are often greater than 2^28.", + "cppType": "uint32", + "csType": "uint", + "goType": "uint32", + "javaType": "int", + "phpType": "integer", + "pythonType": "int", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "fixed64", + "notes": "Always eight bytes. More efficient than uint64 if values are often greater than 2^56.", + "cppType": "uint64", + "csType": "ulong", + "goType": "uint64", + "javaType": "long", + "phpType": "integer/string", + "pythonType": "int/long", + "rubyType": "Bignum" + }, + { + "protoType": "sfixed32", + "notes": "Always four bytes.", + "cppType": "int32", + "csType": "int", + "goType": "int32", + "javaType": "int", + "phpType": "integer", + "pythonType": "int", + "rubyType": "Bignum or Fixnum (as required)" + }, + { + "protoType": "sfixed64", + "notes": "Always eight bytes.", + "cppType": "int64", + "csType": "long", + "goType": "int64", + "javaType": "long", + "phpType": "integer/string", + "pythonType": "int/long", + "rubyType": "Bignum" + }, + { + "protoType": "bool", + "notes": "", + "cppType": "bool", + "csType": "bool", + "goType": "bool", + "javaType": "boolean", + "phpType": "boolean", + "pythonType": "boolean", + "rubyType": "TrueClass/FalseClass" + }, + { + "protoType": "string", + "notes": "A string must always contain UTF-8 encoded or 7-bit ASCII text.", + "cppType": "string", + "csType": "string", + "goType": "string", + "javaType": "String", + "phpType": "string", + "pythonType": "str/unicode", + "rubyType": "String (UTF-8)" + }, + { + "protoType": "bytes", + "notes": "May contain any arbitrary sequence of bytes.", + "cppType": "string", + "csType": "ByteString", + "goType": "[]byte", + "javaType": "ByteString", + "phpType": "string", + "pythonType": "str", + "rubyType": "String (ASCII-8BIT)" + } + ] +} \ No newline at end of file diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts new file mode 100644 index 0000000000..e3a81ef11f --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_connect.ts @@ -0,0 +1,76 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/api_key.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { CreateAccountApiKeyRequest, CreateAccountApiKeyResponse, DeleteAccountApiKeyRequest, DeleteAccountApiKeyResponse, GetAccountApiKeyRequest, GetAccountApiKeyResponse, GetAccountApiKeysRequest, GetAccountApiKeysResponse, RegenerateAccountApiKeyRequest, RegenerateAccountApiKeyResponse } from "./api_key_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * Service that manages the lifecycle of API Keys that are associated with a specific Account. + * + * @generated from service mgmt.v1alpha1.ApiKeyService + */ +export const ApiKeyService = { + typeName: "mgmt.v1alpha1.ApiKeyService", + methods: { + /** + * Retrieves a list of Account API Keys + * + * @generated from rpc mgmt.v1alpha1.ApiKeyService.GetAccountApiKeys + */ + getAccountApiKeys: { + name: "GetAccountApiKeys", + I: GetAccountApiKeysRequest, + O: GetAccountApiKeysResponse, + kind: MethodKind.Unary, + }, + /** + * Retrieves a single API Key + * + * @generated from rpc mgmt.v1alpha1.ApiKeyService.GetAccountApiKey + */ + getAccountApiKey: { + name: "GetAccountApiKey", + I: GetAccountApiKeyRequest, + O: GetAccountApiKeyResponse, + kind: MethodKind.Unary, + }, + /** + * Creates a single API Key + * This method will return the decrypted contents of the API key + * + * @generated from rpc mgmt.v1alpha1.ApiKeyService.CreateAccountApiKey + */ + createAccountApiKey: { + name: "CreateAccountApiKey", + I: CreateAccountApiKeyRequest, + O: CreateAccountApiKeyResponse, + kind: MethodKind.Unary, + }, + /** + * Regenerates a single API Key with a new expiration time + * This method will return the decrypted contents of the API key + * + * @generated from rpc mgmt.v1alpha1.ApiKeyService.RegenerateAccountApiKey + */ + regenerateAccountApiKey: { + name: "RegenerateAccountApiKey", + I: RegenerateAccountApiKeyRequest, + O: RegenerateAccountApiKeyResponse, + kind: MethodKind.Unary, + }, + /** + * Deletes an API Key from the system. + * + * @generated from rpc mgmt.v1alpha1.ApiKeyService.DeleteAccountApiKey + */ + deleteAccountApiKey: { + name: "DeleteAccountApiKey", + I: DeleteAccountApiKeyRequest, + O: DeleteAccountApiKeyResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts new file mode 100644 index 0000000000..0799c06d3d --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/api_key_pb.ts @@ -0,0 +1,491 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/api_key.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; + +/** + * @generated from message mgmt.v1alpha1.CreateAccountApiKeyRequest + */ +export class CreateAccountApiKeyRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * Validate between now and one year: now < x < 365 days + * + * @generated from field: google.protobuf.Timestamp expires_at = 3; + */ + expiresAt?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateAccountApiKeyRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "expires_at", kind: "message", T: Timestamp }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateAccountApiKeyRequest { + return new CreateAccountApiKeyRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateAccountApiKeyRequest { + return new CreateAccountApiKeyRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateAccountApiKeyRequest { + return new CreateAccountApiKeyRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateAccountApiKeyRequest | PlainMessage | undefined, b: CreateAccountApiKeyRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateAccountApiKeyRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateAccountApiKeyResponse + */ +export class CreateAccountApiKeyResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; + */ + apiKey?: AccountApiKey; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateAccountApiKeyResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateAccountApiKeyResponse { + return new CreateAccountApiKeyResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateAccountApiKeyResponse { + return new CreateAccountApiKeyResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateAccountApiKeyResponse { + return new CreateAccountApiKeyResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateAccountApiKeyResponse | PlainMessage | undefined, b: CreateAccountApiKeyResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateAccountApiKeyResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AccountApiKey + */ +export class AccountApiKey extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * The friendly name of the API Key + * + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: string account_id = 3; + */ + accountId = ""; + + /** + * @generated from field: string created_by_id = 4; + */ + createdById = ""; + + /** + * @generated from field: google.protobuf.Timestamp created_at = 5; + */ + createdAt?: Timestamp; + + /** + * @generated from field: string updated_by_id = 6; + */ + updatedById = ""; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 7; + */ + updatedAt?: Timestamp; + + /** + * key_value is only returned on initial creation or when it is regenerated + * + * @generated from field: optional string key_value = 8; + */ + keyValue?: string; + + /** + * @generated from field: string user_id = 9; + */ + userId = ""; + + /** + * The timestamp of what the API key expires and will not longer be usable. + * + * @generated from field: google.protobuf.Timestamp expires_at = 10; + */ + expiresAt?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccountApiKey"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "created_by_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "created_at", kind: "message", T: Timestamp }, + { no: 6, name: "updated_by_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 7, name: "updated_at", kind: "message", T: Timestamp }, + { no: 8, name: "key_value", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 9, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 10, name: "expires_at", kind: "message", T: Timestamp }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccountApiKey { + return new AccountApiKey().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccountApiKey { + return new AccountApiKey().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccountApiKey { + return new AccountApiKey().fromJsonString(jsonString, options); + } + + static equals(a: AccountApiKey | PlainMessage | undefined, b: AccountApiKey | PlainMessage | undefined): boolean { + return proto3.util.equals(AccountApiKey, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountApiKeysRequest + */ +export class GetAccountApiKeysRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeysRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeysRequest { + return new GetAccountApiKeysRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeysRequest { + return new GetAccountApiKeysRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeysRequest { + return new GetAccountApiKeysRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountApiKeysRequest | PlainMessage | undefined, b: GetAccountApiKeysRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountApiKeysRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountApiKeysResponse + */ +export class GetAccountApiKeysResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.AccountApiKey api_keys = 1; + */ + apiKeys: AccountApiKey[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeysResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "api_keys", kind: "message", T: AccountApiKey, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeysResponse { + return new GetAccountApiKeysResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeysResponse { + return new GetAccountApiKeysResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeysResponse { + return new GetAccountApiKeysResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountApiKeysResponse | PlainMessage | undefined, b: GetAccountApiKeysResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountApiKeysResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountApiKeyRequest + */ +export class GetAccountApiKeyRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeyRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeyRequest { + return new GetAccountApiKeyRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeyRequest { + return new GetAccountApiKeyRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeyRequest { + return new GetAccountApiKeyRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountApiKeyRequest | PlainMessage | undefined, b: GetAccountApiKeyRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountApiKeyRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountApiKeyResponse + */ +export class GetAccountApiKeyResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; + */ + apiKey?: AccountApiKey; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountApiKeyResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountApiKeyResponse { + return new GetAccountApiKeyResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountApiKeyResponse { + return new GetAccountApiKeyResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountApiKeyResponse { + return new GetAccountApiKeyResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountApiKeyResponse | PlainMessage | undefined, b: GetAccountApiKeyResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountApiKeyResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RegenerateAccountApiKeyRequest + */ +export class RegenerateAccountApiKeyRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * Validate between now and one year: now < x < 365 days + * + * @generated from field: google.protobuf.Timestamp expires_at = 2; + */ + expiresAt?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RegenerateAccountApiKeyRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "expires_at", kind: "message", T: Timestamp }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RegenerateAccountApiKeyRequest { + return new RegenerateAccountApiKeyRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RegenerateAccountApiKeyRequest { + return new RegenerateAccountApiKeyRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RegenerateAccountApiKeyRequest { + return new RegenerateAccountApiKeyRequest().fromJsonString(jsonString, options); + } + + static equals(a: RegenerateAccountApiKeyRequest | PlainMessage | undefined, b: RegenerateAccountApiKeyRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(RegenerateAccountApiKeyRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RegenerateAccountApiKeyResponse + */ +export class RegenerateAccountApiKeyResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountApiKey api_key = 1; + */ + apiKey?: AccountApiKey; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RegenerateAccountApiKeyResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "api_key", kind: "message", T: AccountApiKey }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RegenerateAccountApiKeyResponse { + return new RegenerateAccountApiKeyResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RegenerateAccountApiKeyResponse { + return new RegenerateAccountApiKeyResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RegenerateAccountApiKeyResponse { + return new RegenerateAccountApiKeyResponse().fromJsonString(jsonString, options); + } + + static equals(a: RegenerateAccountApiKeyResponse | PlainMessage | undefined, b: RegenerateAccountApiKeyResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(RegenerateAccountApiKeyResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteAccountApiKeyRequest + */ +export class DeleteAccountApiKeyRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteAccountApiKeyRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteAccountApiKeyRequest { + return new DeleteAccountApiKeyRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteAccountApiKeyRequest { + return new DeleteAccountApiKeyRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteAccountApiKeyRequest { + return new DeleteAccountApiKeyRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteAccountApiKeyRequest | PlainMessage | undefined, b: DeleteAccountApiKeyRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteAccountApiKeyRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteAccountApiKeyResponse + */ +export class DeleteAccountApiKeyResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteAccountApiKeyResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteAccountApiKeyResponse { + return new DeleteAccountApiKeyResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteAccountApiKeyResponse { + return new DeleteAccountApiKeyResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteAccountApiKeyResponse { + return new DeleteAccountApiKeyResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteAccountApiKeyResponse | PlainMessage | undefined, b: DeleteAccountApiKeyResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteAccountApiKeyResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts new file mode 100644 index 0000000000..dcfdee3054 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_connect.ts @@ -0,0 +1,89 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/auth.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { CheckTokenRequest, CheckTokenResponse, GetAuthorizeUrlRequest, GetAuthorizeUrlResponse, GetAuthStatusRequest, GetAuthStatusResponse, GetCliIssuerRequest, GetCliIssuerResponse, LoginCliRequest, LoginCliResponse, RefreshCliRequest, RefreshCliResponse } from "./auth_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * Service that handles generic Authentication for Neosync + * Today this is mostly used by the CLI to receive authentication information + * + * @generated from service mgmt.v1alpha1.AuthService + */ +export const AuthService = { + typeName: "mgmt.v1alpha1.AuthService", + methods: { + /** + * Used by the CLI to login to Neosync with OAuth. + * + * @generated from rpc mgmt.v1alpha1.AuthService.LoginCli + */ + loginCli: { + name: "LoginCli", + I: LoginCliRequest, + O: LoginCliResponse, + kind: MethodKind.Unary, + }, + /** + * Used by the CLI to refresh an expired Neosync accesss token. + * This should only be used if an access token was previously retrieved from the `LoginCli` or `RefreshCli` methods. + * + * @generated from rpc mgmt.v1alpha1.AuthService.RefreshCli + */ + refreshCli: { + name: "RefreshCli", + I: RefreshCliRequest, + O: RefreshCliResponse, + kind: MethodKind.Unary, + }, + /** + * Empty endpoint to simply check if the provided access token is valid + * + * @generated from rpc mgmt.v1alpha1.AuthService.CheckToken + */ + checkToken: { + name: "CheckToken", + I: CheckTokenRequest, + O: CheckTokenResponse, + kind: MethodKind.Unary, + }, + /** + * Used by the CLI to retrieve Auth Issuer information + * @deprecated + * + * @generated from rpc mgmt.v1alpha1.AuthService.GetCliIssuer + */ + getCliIssuer: { + name: "GetCliIssuer", + I: GetCliIssuerRequest, + O: GetCliIssuerResponse, + kind: MethodKind.Unary, + }, + /** + * Used by the CLI to retrieve an Authorize URL for use with OAuth login. + * + * @generated from rpc mgmt.v1alpha1.AuthService.GetAuthorizeUrl + */ + getAuthorizeUrl: { + name: "GetAuthorizeUrl", + I: GetAuthorizeUrlRequest, + O: GetAuthorizeUrlResponse, + kind: MethodKind.Unary, + }, + /** + * Returns the auth status of the API server. Whether or not the backend has authentication enabled. + * This is used by clients to make decisions on whether or not they should send access tokens to the API. + * + * @generated from rpc mgmt.v1alpha1.AuthService.GetAuthStatus + */ + getAuthStatus: { + name: "GetAuthStatus", + I: GetAuthStatusRequest, + O: GetAuthStatusResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts new file mode 100644 index 0000000000..0a69262c0c --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/auth_pb.ts @@ -0,0 +1,559 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/auth.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, protoInt64 } from "@bufbuild/protobuf"; + +/** + * @generated from message mgmt.v1alpha1.LoginCliRequest + */ +export class LoginCliRequest extends Message { + /** + * The oauth code + * + * @generated from field: string code = 1; + */ + code = ""; + + /** + * The oauth redirect uri that the client uses during the oauth request + * + * @generated from field: string redirect_uri = 2; + */ + redirectUri = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.LoginCliRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "redirect_uri", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): LoginCliRequest { + return new LoginCliRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): LoginCliRequest { + return new LoginCliRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): LoginCliRequest { + return new LoginCliRequest().fromJsonString(jsonString, options); + } + + static equals(a: LoginCliRequest | PlainMessage | undefined, b: LoginCliRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(LoginCliRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.LoginCliResponse + */ +export class LoginCliResponse extends Message { + /** + * The access token that is returned on successful login + * + * @generated from field: mgmt.v1alpha1.AccessToken access_token = 1; + */ + accessToken?: AccessToken; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.LoginCliResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "access_token", kind: "message", T: AccessToken }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): LoginCliResponse { + return new LoginCliResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): LoginCliResponse { + return new LoginCliResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): LoginCliResponse { + return new LoginCliResponse().fromJsonString(jsonString, options); + } + + static equals(a: LoginCliResponse | PlainMessage | undefined, b: LoginCliResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(LoginCliResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAuthStatusRequest + */ +export class GetAuthStatusRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAuthStatusRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthStatusRequest { + return new GetAuthStatusRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthStatusRequest { + return new GetAuthStatusRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAuthStatusRequest { + return new GetAuthStatusRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAuthStatusRequest | PlainMessage | undefined, b: GetAuthStatusRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAuthStatusRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAuthStatusResponse + */ +export class GetAuthStatusResponse extends Message { + /** + * Whether or not the server has authentication enabled. + * This tells the client if it is expected to send access tokens. + * + * @generated from field: bool is_enabled = 1; + */ + isEnabled = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAuthStatusResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_enabled", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthStatusResponse { + return new GetAuthStatusResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthStatusResponse { + return new GetAuthStatusResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAuthStatusResponse { + return new GetAuthStatusResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAuthStatusResponse | PlainMessage | undefined, b: GetAuthStatusResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAuthStatusResponse, a, b); + } +} + +/** + * A decoded representation of an Access token from the backing auth server + * + * @generated from message mgmt.v1alpha1.AccessToken + */ +export class AccessToken extends Message { + /** + * The access token that will be provided in subsequent requests to provide authenticated access to the Api + * + * @generated from field: string access_token = 1; + */ + accessToken = ""; + + /** + * Token that can be used to retrieve a refreshed access token. + * Will not be provided if the offline_access scope is not provided in the initial login flow. + * + * @generated from field: optional string refresh_token = 2; + */ + refreshToken?: string; + + /** + * Relative time in seconds that the access token will expire. Combine with the current time to get the expires_at time. + * + * @generated from field: int64 expires_in = 3; + */ + expiresIn = protoInt64.zero; + + /** + * The scopes that the access token have + * + * @generated from field: string scope = 4; + */ + scope = ""; + + /** + * The identity token of the authenticated user + * + * @generated from field: optional string id_token = 5; + */ + idToken?: string; + + /** + * The token type. For JWTs, this will be `Bearer` + * + * @generated from field: string token_type = 6; + */ + tokenType = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccessToken"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "access_token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "refresh_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 3, name: "expires_in", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 4, name: "scope", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "id_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 6, name: "token_type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccessToken { + return new AccessToken().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccessToken { + return new AccessToken().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccessToken { + return new AccessToken().fromJsonString(jsonString, options); + } + + static equals(a: AccessToken | PlainMessage | undefined, b: AccessToken | PlainMessage | undefined): boolean { + return proto3.util.equals(AccessToken, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAuthorizeUrlRequest + */ +export class GetAuthorizeUrlRequest extends Message { + /** + * The state that's generated by the client that is passed along to prevent tampering + * + * @generated from field: string state = 1; + */ + state = ""; + + /** + * The redirect uri that the client will be redirected back to during the auth request + * + * @generated from field: string redirect_uri = 2; + */ + redirectUri = ""; + + /** + * The scopes the client is requesting as a part of the oauth login request + * + * @generated from field: string scope = 3; + */ + scope = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAuthorizeUrlRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "state", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "redirect_uri", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "scope", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthorizeUrlRequest { + return new GetAuthorizeUrlRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthorizeUrlRequest { + return new GetAuthorizeUrlRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAuthorizeUrlRequest { + return new GetAuthorizeUrlRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAuthorizeUrlRequest | PlainMessage | undefined, b: GetAuthorizeUrlRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAuthorizeUrlRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAuthorizeUrlResponse + */ +export class GetAuthorizeUrlResponse extends Message { + /** + * The generated url that is the client will be redirected to during the Oauth flow + * + * @generated from field: string url = 1; + */ + url = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAuthorizeUrlResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAuthorizeUrlResponse { + return new GetAuthorizeUrlResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAuthorizeUrlResponse { + return new GetAuthorizeUrlResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAuthorizeUrlResponse { + return new GetAuthorizeUrlResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAuthorizeUrlResponse | PlainMessage | undefined, b: GetAuthorizeUrlResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAuthorizeUrlResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetCliIssuerRequest + */ +export class GetCliIssuerRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetCliIssuerRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetCliIssuerRequest { + return new GetCliIssuerRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetCliIssuerRequest { + return new GetCliIssuerRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetCliIssuerRequest { + return new GetCliIssuerRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetCliIssuerRequest | PlainMessage | undefined, b: GetCliIssuerRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetCliIssuerRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetCliIssuerResponse + */ +export class GetCliIssuerResponse extends Message { + /** + * The backing authentication issuer url + * + * @generated from field: string issuer_url = 1; + */ + issuerUrl = ""; + + /** + * The audience that will be used in the access token. This corresponds to the "aud" claim + * + * @generated from field: string audience = 2; + */ + audience = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetCliIssuerResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "issuer_url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "audience", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetCliIssuerResponse { + return new GetCliIssuerResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetCliIssuerResponse { + return new GetCliIssuerResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetCliIssuerResponse { + return new GetCliIssuerResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetCliIssuerResponse | PlainMessage | undefined, b: GetCliIssuerResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetCliIssuerResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RefreshCliRequest + */ +export class RefreshCliRequest extends Message { + /** + * The token used to retrieve a new access token. + * + * @generated from field: string refresh_token = 1; + */ + refreshToken = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RefreshCliRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "refresh_token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RefreshCliRequest { + return new RefreshCliRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RefreshCliRequest { + return new RefreshCliRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RefreshCliRequest { + return new RefreshCliRequest().fromJsonString(jsonString, options); + } + + static equals(a: RefreshCliRequest | PlainMessage | undefined, b: RefreshCliRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(RefreshCliRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RefreshCliResponse + */ +export class RefreshCliResponse extends Message { + /** + * The access token that is returned on successful refresh + * + * @generated from field: mgmt.v1alpha1.AccessToken access_token = 1; + */ + accessToken?: AccessToken; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RefreshCliResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "access_token", kind: "message", T: AccessToken }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RefreshCliResponse { + return new RefreshCliResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RefreshCliResponse { + return new RefreshCliResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RefreshCliResponse { + return new RefreshCliResponse().fromJsonString(jsonString, options); + } + + static equals(a: RefreshCliResponse | PlainMessage | undefined, b: RefreshCliResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(RefreshCliResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckTokenRequest + */ +export class CheckTokenRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckTokenRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckTokenRequest { + return new CheckTokenRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckTokenRequest { + return new CheckTokenRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckTokenRequest { + return new CheckTokenRequest().fromJsonString(jsonString, options); + } + + static equals(a: CheckTokenRequest | PlainMessage | undefined, b: CheckTokenRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckTokenRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckTokenResponse + */ +export class CheckTokenResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckTokenResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckTokenResponse { + return new CheckTokenResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckTokenResponse { + return new CheckTokenResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckTokenResponse { + return new CheckTokenResponse().fromJsonString(jsonString, options); + } + + static equals(a: CheckTokenResponse | PlainMessage | undefined, b: CheckTokenResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckTokenResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts new file mode 100644 index 0000000000..3d3720b51a --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_connect.ts @@ -0,0 +1,110 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/connection.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { CheckConnectionConfigRequest, CheckConnectionConfigResponse, CheckSqlQueryRequest, CheckSqlQueryResponse, CreateConnectionRequest, CreateConnectionResponse, DeleteConnectionRequest, DeleteConnectionResponse, GetConnectionRequest, GetConnectionResponse, GetConnectionsRequest, GetConnectionsResponse, IsConnectionNameAvailableRequest, IsConnectionNameAvailableResponse, UpdateConnectionRequest, UpdateConnectionResponse } from "./connection_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * Service for managing datasource connections. + * This is a primary data model in Neosync and is used in reference when hooking up Jobs to synchronize and generate data. + * + * @generated from service mgmt.v1alpha1.ConnectionService + */ +export const ConnectionService = { + typeName: "mgmt.v1alpha1.ConnectionService", + methods: { + /** + * Returns a list of connections associated with the account + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.GetConnections + */ + getConnections: { + name: "GetConnections", + I: GetConnectionsRequest, + O: GetConnectionsResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a single connection + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.GetConnection + */ + getConnection: { + name: "GetConnection", + I: GetConnectionRequest, + O: GetConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * Creates a new connection + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.CreateConnection + */ + createConnection: { + name: "CreateConnection", + I: CreateConnectionRequest, + O: CreateConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * Updates an existing connection + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.UpdateConnection + */ + updateConnection: { + name: "UpdateConnection", + I: UpdateConnectionRequest, + O: UpdateConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * Removes a connection from the system. + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.DeleteConnection + */ + deleteConnection: { + name: "DeleteConnection", + I: DeleteConnectionRequest, + O: DeleteConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * Connections have friendly names, this method checks if the requested name is available in the system based on the account + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.IsConnectionNameAvailable + */ + isConnectionNameAvailable: { + name: "IsConnectionNameAvailable", + I: IsConnectionNameAvailableRequest, + O: IsConnectionNameAvailableResponse, + kind: MethodKind.Unary, + }, + /** + * Checks if the connection config is connectable by the backend. + * Used mostly to verify that a connection is valid prior to creating a Connection object. + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.CheckConnectionConfig + */ + checkConnectionConfig: { + name: "CheckConnectionConfig", + I: CheckConnectionConfigRequest, + O: CheckConnectionConfigResponse, + kind: MethodKind.Unary, + }, + /** + * Checks a constructed SQL query against a sql-based connection to see if it's valid based on that connection's data schema + * This is useful when constructing subsets to see if the WHERE clause is correct + * + * @generated from rpc mgmt.v1alpha1.ConnectionService.CheckSqlQuery + */ + checkSqlQuery: { + name: "CheckSqlQuery", + I: CheckSqlQueryRequest, + O: CheckSqlQueryResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts new file mode 100644 index 0000000000..e4cd5329e6 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_connect.ts @@ -0,0 +1,90 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/connection_data.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { GetConnectionDataStreamRequest, GetConnectionDataStreamResponse, GetConnectionForeignConstraintsRequest, GetConnectionForeignConstraintsResponse, GetConnectionInitStatementsRequest, GetConnectionInitStatementsResponse, GetConnectionPrimaryConstraintsRequest, GetConnectionPrimaryConstraintsResponse, GetConnectionSchemaRequest, GetConnectionSchemaResponse, GetConnectionUniqueConstraintsRequest, GetConnectionUniqueConstraintsResponse } from "./connection_data_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * Service for managing connection data. + * This is used in handle data from a connection + * + * @generated from service mgmt.v1alpha1.ConnectionDataService + */ +export const ConnectionDataService = { + typeName: "mgmt.v1alpha1.ConnectionDataService", + methods: { + /** + * Streaming endpoint that will stream the data available from the Connection to the client. + * Used primarily by the CLI sync command. + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionDataStream + */ + getConnectionDataStream: { + name: "GetConnectionDataStream", + I: GetConnectionDataStreamRequest, + O: GetConnectionDataStreamResponse, + kind: MethodKind.ServerStreaming, + }, + /** + * Returns the schema for a specific connection. Used mostly for SQL-based connections + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionSchema + */ + getConnectionSchema: { + name: "GetConnectionSchema", + I: GetConnectionSchemaRequest, + O: GetConnectionSchemaResponse, + kind: MethodKind.Unary, + }, + /** + * For a specific connection, returns the foreign key constraints. Mostly useful for SQL-based Connections. + * Used primarily by the CLI sync command to determine stream order. + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionForeignConstraints + */ + getConnectionForeignConstraints: { + name: "GetConnectionForeignConstraints", + I: GetConnectionForeignConstraintsRequest, + O: GetConnectionForeignConstraintsResponse, + kind: MethodKind.Unary, + }, + /** + * For a specific connection, returns the primary key constraints. Mostly useful for SQL-based Connections. + * Used primarily by the CLI sync command to determine stream order. + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionPrimaryConstraints + */ + getConnectionPrimaryConstraints: { + name: "GetConnectionPrimaryConstraints", + I: GetConnectionPrimaryConstraintsRequest, + O: GetConnectionPrimaryConstraintsResponse, + kind: MethodKind.Unary, + }, + /** + * For a specific connection, returns the init table statements. Mostly useful for SQL-based Connections. + * Used primarily by the CLI sync command to create table schema init statement. + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionInitStatements + */ + getConnectionInitStatements: { + name: "GetConnectionInitStatements", + I: GetConnectionInitStatementsRequest, + O: GetConnectionInitStatementsResponse, + kind: MethodKind.Unary, + }, + /** + * For a specific connection, returns the unique constraints. Mostly useful for SQL-based connections. + * + * @generated from rpc mgmt.v1alpha1.ConnectionDataService.GetConnectionUniqueConstraints + */ + getConnectionUniqueConstraints: { + name: "GetConnectionUniqueConstraints", + I: GetConnectionUniqueConstraintsRequest, + O: GetConnectionUniqueConstraintsResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts new file mode 100644 index 0000000000..142806ac5d --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_data_pb.ts @@ -0,0 +1,1171 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/connection_data.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3 } from "@bufbuild/protobuf"; + +/** + * @generated from message mgmt.v1alpha1.PostgresStreamConfig + */ +export class PostgresStreamConfig extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresStreamConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresStreamConfig { + return new PostgresStreamConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresStreamConfig { + return new PostgresStreamConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresStreamConfig { + return new PostgresStreamConfig().fromJsonString(jsonString, options); + } + + static equals(a: PostgresStreamConfig | PlainMessage | undefined, b: PostgresStreamConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresStreamConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlStreamConfig + */ +export class MysqlStreamConfig extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlStreamConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlStreamConfig { + return new MysqlStreamConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlStreamConfig { + return new MysqlStreamConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlStreamConfig { + return new MysqlStreamConfig().fromJsonString(jsonString, options); + } + + static equals(a: MysqlStreamConfig | PlainMessage | undefined, b: MysqlStreamConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlStreamConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AwsS3StreamConfig + */ +export class AwsS3StreamConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.AwsS3StreamConfig.id + */ + id: { + /** + * @generated from field: string job_id = 1; + */ + value: string; + case: "jobId"; + } | { + /** + * @generated from field: string job_run_id = 2; + */ + value: string; + case: "jobRunId"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3StreamConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3StreamConfig { + return new AwsS3StreamConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3StreamConfig { + return new AwsS3StreamConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3StreamConfig { + return new AwsS3StreamConfig().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3StreamConfig | PlainMessage | undefined, b: AwsS3StreamConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3StreamConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConnectionStreamConfig + */ +export class ConnectionStreamConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.ConnectionStreamConfig.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.PostgresStreamConfig pg_config = 1; + */ + value: PostgresStreamConfig; + case: "pgConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.AwsS3StreamConfig aws_s3_config = 2; + */ + value: AwsS3StreamConfig; + case: "awsS3Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlStreamConfig mysql_config = 3; + */ + value: MysqlStreamConfig; + case: "mysqlConfig"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConnectionStreamConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "pg_config", kind: "message", T: PostgresStreamConfig, oneof: "config" }, + { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3StreamConfig, oneof: "config" }, + { no: 3, name: "mysql_config", kind: "message", T: MysqlStreamConfig, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionStreamConfig { + return new ConnectionStreamConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionStreamConfig { + return new ConnectionStreamConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConnectionStreamConfig { + return new ConnectionStreamConfig().fromJsonString(jsonString, options); + } + + static equals(a: ConnectionStreamConfig | PlainMessage | undefined, b: ConnectionStreamConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(ConnectionStreamConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionDataStreamRequest + */ +export class GetConnectionDataStreamRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.ConnectionStreamConfig stream_config = 2; + */ + streamConfig?: ConnectionStreamConfig; + + /** + * @generated from field: string schema = 3; + */ + schema = ""; + + /** + * @generated from field: string table = 4; + */ + table = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionDataStreamRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "stream_config", kind: "message", T: ConnectionStreamConfig }, + { no: 3, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionDataStreamRequest { + return new GetConnectionDataStreamRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionDataStreamRequest { + return new GetConnectionDataStreamRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionDataStreamRequest { + return new GetConnectionDataStreamRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionDataStreamRequest | PlainMessage | undefined, b: GetConnectionDataStreamRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionDataStreamRequest, a, b); + } +} + +/** + * Each stream response is a single row in the requested schema and table + * + * @generated from message mgmt.v1alpha1.GetConnectionDataStreamResponse + */ +export class GetConnectionDataStreamResponse extends Message { + /** + * A map of column name to the bytes value of the data that was found for that column and row + * + * @generated from field: map row = 1; + */ + row: { [key: string]: Uint8Array } = {}; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionDataStreamResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "row", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 12 /* ScalarType.BYTES */} }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionDataStreamResponse { + return new GetConnectionDataStreamResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionDataStreamResponse { + return new GetConnectionDataStreamResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionDataStreamResponse { + return new GetConnectionDataStreamResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionDataStreamResponse | PlainMessage | undefined, b: GetConnectionDataStreamResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionDataStreamResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresSchemaConfig + */ +export class PostgresSchemaConfig extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresSchemaConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSchemaConfig { + return new PostgresSchemaConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSchemaConfig { + return new PostgresSchemaConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresSchemaConfig { + return new PostgresSchemaConfig().fromJsonString(jsonString, options); + } + + static equals(a: PostgresSchemaConfig | PlainMessage | undefined, b: PostgresSchemaConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresSchemaConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlSchemaConfig + */ +export class MysqlSchemaConfig extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlSchemaConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSchemaConfig { + return new MysqlSchemaConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSchemaConfig { + return new MysqlSchemaConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlSchemaConfig { + return new MysqlSchemaConfig().fromJsonString(jsonString, options); + } + + static equals(a: MysqlSchemaConfig | PlainMessage | undefined, b: MysqlSchemaConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlSchemaConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AwsS3SchemaConfig + */ +export class AwsS3SchemaConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.AwsS3SchemaConfig.id + */ + id: { + /** + * @generated from field: string job_id = 1; + */ + value: string; + case: "jobId"; + } | { + /** + * @generated from field: string job_run_id = 2; + */ + value: string; + case: "jobRunId"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3SchemaConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3SchemaConfig { + return new AwsS3SchemaConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3SchemaConfig { + return new AwsS3SchemaConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3SchemaConfig { + return new AwsS3SchemaConfig().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3SchemaConfig | PlainMessage | undefined, b: AwsS3SchemaConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3SchemaConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConnectionSchemaConfig + */ +export class ConnectionSchemaConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.ConnectionSchemaConfig.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.PostgresSchemaConfig pg_config = 1; + */ + value: PostgresSchemaConfig; + case: "pgConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.AwsS3SchemaConfig aws_s3_config = 2; + */ + value: AwsS3SchemaConfig; + case: "awsS3Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlSchemaConfig mysql_config = 3; + */ + value: MysqlSchemaConfig; + case: "mysqlConfig"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConnectionSchemaConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "pg_config", kind: "message", T: PostgresSchemaConfig, oneof: "config" }, + { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3SchemaConfig, oneof: "config" }, + { no: 3, name: "mysql_config", kind: "message", T: MysqlSchemaConfig, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionSchemaConfig { + return new ConnectionSchemaConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionSchemaConfig { + return new ConnectionSchemaConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConnectionSchemaConfig { + return new ConnectionSchemaConfig().fromJsonString(jsonString, options); + } + + static equals(a: ConnectionSchemaConfig | PlainMessage | undefined, b: ConnectionSchemaConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(ConnectionSchemaConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DatabaseColumn + */ +export class DatabaseColumn extends Message { + /** + * The database schema. Ex: public + * + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * The name of the table in the schema + * + * @generated from field: string table = 2; + */ + table = ""; + + /** + * The name of the column + * + * @generated from field: string column = 3; + */ + column = ""; + + /** + * The datatype of the column + * + * @generated from field: string data_type = 4; + */ + dataType = ""; + + /** + * The isNullable Flag of the column + * + * @generated from field: string is_nullable = 5; + */ + isNullable = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DatabaseColumn"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "data_type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "is_nullable", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DatabaseColumn { + return new DatabaseColumn().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DatabaseColumn { + return new DatabaseColumn().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DatabaseColumn { + return new DatabaseColumn().fromJsonString(jsonString, options); + } + + static equals(a: DatabaseColumn | PlainMessage | undefined, b: DatabaseColumn | PlainMessage | undefined): boolean { + return proto3.util.equals(DatabaseColumn, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionSchemaRequest + */ +export class GetConnectionSchemaRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.ConnectionSchemaConfig schema_config = 2; + */ + schemaConfig?: ConnectionSchemaConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionSchemaRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "schema_config", kind: "message", T: ConnectionSchemaConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionSchemaRequest { + return new GetConnectionSchemaRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionSchemaRequest { + return new GetConnectionSchemaRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionSchemaRequest { + return new GetConnectionSchemaRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionSchemaRequest | PlainMessage | undefined, b: GetConnectionSchemaRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionSchemaRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionSchemaResponse + */ +export class GetConnectionSchemaResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.DatabaseColumn schemas = 1; + */ + schemas: DatabaseColumn[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionSchemaResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schemas", kind: "message", T: DatabaseColumn, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionSchemaResponse { + return new GetConnectionSchemaResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionSchemaResponse { + return new GetConnectionSchemaResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionSchemaResponse { + return new GetConnectionSchemaResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionSchemaResponse | PlainMessage | undefined, b: GetConnectionSchemaResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionSchemaResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionForeignConstraintsRequest + */ +export class GetConnectionForeignConstraintsRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionForeignConstraintsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionForeignConstraintsRequest { + return new GetConnectionForeignConstraintsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionForeignConstraintsRequest { + return new GetConnectionForeignConstraintsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionForeignConstraintsRequest { + return new GetConnectionForeignConstraintsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionForeignConstraintsRequest | PlainMessage | undefined, b: GetConnectionForeignConstraintsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionForeignConstraintsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ForeignKey + */ +export class ForeignKey extends Message { + /** + * @generated from field: string table = 1; + */ + table = ""; + + /** + * @generated from field: string column = 2; + */ + column = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ForeignKey"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ForeignKey { + return new ForeignKey().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ForeignKey { + return new ForeignKey().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ForeignKey { + return new ForeignKey().fromJsonString(jsonString, options); + } + + static equals(a: ForeignKey | PlainMessage | undefined, b: ForeignKey | PlainMessage | undefined): boolean { + return proto3.util.equals(ForeignKey, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ForeignConstraint + */ +export class ForeignConstraint extends Message { + /** + * @generated from field: string column = 1; + */ + column = ""; + + /** + * @generated from field: bool is_nullable = 2; + */ + isNullable = false; + + /** + * @generated from field: mgmt.v1alpha1.ForeignKey foreign_key = 3; + */ + foreignKey?: ForeignKey; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ForeignConstraint"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "is_nullable", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "foreign_key", kind: "message", T: ForeignKey }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ForeignConstraint { + return new ForeignConstraint().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ForeignConstraint { + return new ForeignConstraint().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ForeignConstraint { + return new ForeignConstraint().fromJsonString(jsonString, options); + } + + static equals(a: ForeignConstraint | PlainMessage | undefined, b: ForeignConstraint | PlainMessage | undefined): boolean { + return proto3.util.equals(ForeignConstraint, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ForeignConstraintTables + */ +export class ForeignConstraintTables extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.ForeignConstraint constraints = 1; + */ + constraints: ForeignConstraint[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ForeignConstraintTables"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "constraints", kind: "message", T: ForeignConstraint, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ForeignConstraintTables { + return new ForeignConstraintTables().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ForeignConstraintTables { + return new ForeignConstraintTables().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ForeignConstraintTables { + return new ForeignConstraintTables().fromJsonString(jsonString, options); + } + + static equals(a: ForeignConstraintTables | PlainMessage | undefined, b: ForeignConstraintTables | PlainMessage | undefined): boolean { + return proto3.util.equals(ForeignConstraintTables, a, b); + } +} + +/** + * Dependency constraints for a specific table + * + * @generated from message mgmt.v1alpha1.GetConnectionForeignConstraintsResponse + */ +export class GetConnectionForeignConstraintsResponse extends Message { + /** + * the key here is .
and the list of tables that it depends on, also `.
` format. + * + * @generated from field: map table_constraints = 1; + */ + tableConstraints: { [key: string]: ForeignConstraintTables } = {}; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionForeignConstraintsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: ForeignConstraintTables} }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionForeignConstraintsResponse { + return new GetConnectionForeignConstraintsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionForeignConstraintsResponse { + return new GetConnectionForeignConstraintsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionForeignConstraintsResponse { + return new GetConnectionForeignConstraintsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionForeignConstraintsResponse | PlainMessage | undefined, b: GetConnectionForeignConstraintsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionForeignConstraintsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.InitStatementOptions + */ +export class InitStatementOptions extends Message { + /** + * @generated from field: bool init_schema = 1; + */ + initSchema = false; + + /** + * @generated from field: bool truncate_before_insert = 2; + */ + truncateBeforeInsert = false; + + /** + * @generated from field: bool truncate_cascade = 3; + */ + truncateCascade = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.InitStatementOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "init_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "truncate_cascade", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): InitStatementOptions { + return new InitStatementOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): InitStatementOptions { + return new InitStatementOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): InitStatementOptions { + return new InitStatementOptions().fromJsonString(jsonString, options); + } + + static equals(a: InitStatementOptions | PlainMessage | undefined, b: InitStatementOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(InitStatementOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionInitStatementsRequest + */ +export class GetConnectionInitStatementsRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.InitStatementOptions options = 2; + */ + options?: InitStatementOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionInitStatementsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "options", kind: "message", T: InitStatementOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionInitStatementsRequest { + return new GetConnectionInitStatementsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionInitStatementsRequest { + return new GetConnectionInitStatementsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionInitStatementsRequest { + return new GetConnectionInitStatementsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionInitStatementsRequest | PlainMessage | undefined, b: GetConnectionInitStatementsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionInitStatementsRequest, a, b); + } +} + +/** + * Init statement for a specific table + * + * @generated from message mgmt.v1alpha1.GetConnectionInitStatementsResponse + */ +export class GetConnectionInitStatementsResponse extends Message { + /** + * the key here is .
and value is the table init statement. + * + * @generated from field: map table_init_statements = 1; + */ + tableInitStatements: { [key: string]: string } = {}; + + /** + * the key here is .
and value is the table truncate statement. + * + * @generated from field: map table_truncate_statements = 2; + */ + tableTruncateStatements: { [key: string]: string } = {}; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionInitStatementsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table_init_statements", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 9 /* ScalarType.STRING */} }, + { no: 2, name: "table_truncate_statements", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "scalar", T: 9 /* ScalarType.STRING */} }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionInitStatementsResponse { + return new GetConnectionInitStatementsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionInitStatementsResponse { + return new GetConnectionInitStatementsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionInitStatementsResponse { + return new GetConnectionInitStatementsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionInitStatementsResponse | PlainMessage | undefined, b: GetConnectionInitStatementsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionInitStatementsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PrimaryConstraint + */ +export class PrimaryConstraint extends Message { + /** + * @generated from field: repeated string columns = 1; + */ + columns: string[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PrimaryConstraint"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "columns", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PrimaryConstraint { + return new PrimaryConstraint().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PrimaryConstraint { + return new PrimaryConstraint().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PrimaryConstraint { + return new PrimaryConstraint().fromJsonString(jsonString, options); + } + + static equals(a: PrimaryConstraint | PlainMessage | undefined, b: PrimaryConstraint | PlainMessage | undefined): boolean { + return proto3.util.equals(PrimaryConstraint, a, b); + } +} + +/** + * Primary constraints for a specific table + * + * @generated from message mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest + */ +export class GetConnectionPrimaryConstraintsRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionPrimaryConstraintsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionPrimaryConstraintsRequest { + return new GetConnectionPrimaryConstraintsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionPrimaryConstraintsRequest { + return new GetConnectionPrimaryConstraintsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionPrimaryConstraintsRequest { + return new GetConnectionPrimaryConstraintsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionPrimaryConstraintsRequest | PlainMessage | undefined, b: GetConnectionPrimaryConstraintsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionPrimaryConstraintsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse + */ +export class GetConnectionPrimaryConstraintsResponse extends Message { + /** + * the key here is .
and value is the primary constraint + * + * @generated from field: map table_constraints = 1; + */ + tableConstraints: { [key: string]: PrimaryConstraint } = {}; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionPrimaryConstraintsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: PrimaryConstraint} }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionPrimaryConstraintsResponse { + return new GetConnectionPrimaryConstraintsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionPrimaryConstraintsResponse { + return new GetConnectionPrimaryConstraintsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionPrimaryConstraintsResponse { + return new GetConnectionPrimaryConstraintsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionPrimaryConstraintsResponse | PlainMessage | undefined, b: GetConnectionPrimaryConstraintsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionPrimaryConstraintsResponse, a, b); + } +} + +/** + * Unique constraints for a specific table + * + * @generated from message mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest + */ +export class GetConnectionUniqueConstraintsRequest extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionUniqueConstraintsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionUniqueConstraintsRequest { + return new GetConnectionUniqueConstraintsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionUniqueConstraintsRequest { + return new GetConnectionUniqueConstraintsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionUniqueConstraintsRequest { + return new GetConnectionUniqueConstraintsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionUniqueConstraintsRequest | PlainMessage | undefined, b: GetConnectionUniqueConstraintsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionUniqueConstraintsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse + */ +export class GetConnectionUniqueConstraintsResponse extends Message { + /** + * the key here is .
and value is the unique constraint + * + * @generated from field: map table_constraints = 1; + */ + tableConstraints: { [key: string]: UniqueConstraint } = {}; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionUniqueConstraintsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table_constraints", kind: "map", K: 9 /* ScalarType.STRING */, V: {kind: "message", T: UniqueConstraint} }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionUniqueConstraintsResponse { + return new GetConnectionUniqueConstraintsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionUniqueConstraintsResponse { + return new GetConnectionUniqueConstraintsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionUniqueConstraintsResponse { + return new GetConnectionUniqueConstraintsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionUniqueConstraintsResponse | PlainMessage | undefined, b: GetConnectionUniqueConstraintsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionUniqueConstraintsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UniqueConstraint + */ +export class UniqueConstraint extends Message { + /** + * @generated from field: repeated string columns = 1; + */ + columns: string[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UniqueConstraint"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "columns", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UniqueConstraint { + return new UniqueConstraint().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UniqueConstraint { + return new UniqueConstraint().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UniqueConstraint { + return new UniqueConstraint().fromJsonString(jsonString, options); + } + + static equals(a: UniqueConstraint | PlainMessage | undefined, b: UniqueConstraint | PlainMessage | undefined): boolean { + return proto3.util.equals(UniqueConstraint, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts new file mode 100644 index 0000000000..53530d947c --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/connection_pb.ts @@ -0,0 +1,1575 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/connection.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; + +/** + * @generated from message mgmt.v1alpha1.GetConnectionsRequest + */ +export class GetConnectionsRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionsRequest { + return new GetConnectionsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionsRequest { + return new GetConnectionsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionsRequest { + return new GetConnectionsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionsRequest | PlainMessage | undefined, b: GetConnectionsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionsResponse + */ +export class GetConnectionsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.Connection connections = 1; + */ + connections: Connection[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connections", kind: "message", T: Connection, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionsResponse { + return new GetConnectionsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionsResponse { + return new GetConnectionsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionsResponse { + return new GetConnectionsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionsResponse | PlainMessage | undefined, b: GetConnectionsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionRequest + */ +export class GetConnectionRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionRequest { + return new GetConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionRequest { + return new GetConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionRequest { + return new GetConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionRequest | PlainMessage | undefined, b: GetConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetConnectionResponse + */ +export class GetConnectionResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Connection connection = 1; + */ + connection?: Connection; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection", kind: "message", T: Connection }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetConnectionResponse { + return new GetConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetConnectionResponse { + return new GetConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetConnectionResponse { + return new GetConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetConnectionResponse | PlainMessage | undefined, b: GetConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateConnectionRequest + */ +export class CreateConnectionRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * The friendly name of the connection + * + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; + */ + connectionConfig?: ConnectionConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateConnectionRequest { + return new CreateConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateConnectionRequest { + return new CreateConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateConnectionRequest { + return new CreateConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateConnectionRequest | PlainMessage | undefined, b: CreateConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateConnectionResponse + */ +export class CreateConnectionResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Connection connection = 1; + */ + connection?: Connection; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection", kind: "message", T: Connection }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateConnectionResponse { + return new CreateConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateConnectionResponse { + return new CreateConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateConnectionResponse { + return new CreateConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateConnectionResponse | PlainMessage | undefined, b: CreateConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateConnectionRequest + */ +export class UpdateConnectionRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; + */ + connectionConfig?: ConnectionConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateConnectionRequest { + return new UpdateConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateConnectionRequest { + return new UpdateConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateConnectionRequest { + return new UpdateConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: UpdateConnectionRequest | PlainMessage | undefined, b: UpdateConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateConnectionResponse + */ +export class UpdateConnectionResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Connection connection = 1; + */ + connection?: Connection; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection", kind: "message", T: Connection }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateConnectionResponse { + return new UpdateConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateConnectionResponse { + return new UpdateConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateConnectionResponse { + return new UpdateConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: UpdateConnectionResponse | PlainMessage | undefined, b: UpdateConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteConnectionRequest + */ +export class DeleteConnectionRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteConnectionRequest { + return new DeleteConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteConnectionRequest { + return new DeleteConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteConnectionRequest { + return new DeleteConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteConnectionRequest | PlainMessage | undefined, b: DeleteConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteConnectionResponse + */ +export class DeleteConnectionResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteConnectionResponse { + return new DeleteConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteConnectionResponse { + return new DeleteConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteConnectionResponse { + return new DeleteConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteConnectionResponse | PlainMessage | undefined, b: DeleteConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckConnectionConfigRequest + */ +export class CheckConnectionConfigRequest extends Message { + /** + * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 1; + */ + connectionConfig?: ConnectionConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckConnectionConfigRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_config", kind: "message", T: ConnectionConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckConnectionConfigRequest { + return new CheckConnectionConfigRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckConnectionConfigRequest { + return new CheckConnectionConfigRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckConnectionConfigRequest { + return new CheckConnectionConfigRequest().fromJsonString(jsonString, options); + } + + static equals(a: CheckConnectionConfigRequest | PlainMessage | undefined, b: CheckConnectionConfigRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckConnectionConfigRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckConnectionConfigResponse + */ +export class CheckConnectionConfigResponse extends Message { + /** + * Whether or not the API was able to ping the connection + * + * @generated from field: bool is_connected = 1; + */ + isConnected = false; + + /** + * This is the error that was received if the API was unable to connect + * + * @generated from field: optional string connection_error = 2; + */ + connectionError?: string; + + /** + * @generated from field: repeated mgmt.v1alpha1.ConnectionRolePrivilege privileges = 3; + */ + privileges: ConnectionRolePrivilege[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckConnectionConfigResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_connected", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "connection_error", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 3, name: "privileges", kind: "message", T: ConnectionRolePrivilege, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckConnectionConfigResponse { + return new CheckConnectionConfigResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckConnectionConfigResponse { + return new CheckConnectionConfigResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckConnectionConfigResponse { + return new CheckConnectionConfigResponse().fromJsonString(jsonString, options); + } + + static equals(a: CheckConnectionConfigResponse | PlainMessage | undefined, b: CheckConnectionConfigResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckConnectionConfigResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConnectionRolePrivilege + */ +export class ConnectionRolePrivilege extends Message { + /** + * The role that was given the permissions + * + * @generated from field: string grantee = 1; + */ + grantee = ""; + + /** + * The database schema. Ex: public + * + * @generated from field: string schema = 2; + */ + schema = ""; + + /** + * The name of the table in the schema + * + * @generated from field: string table = 3; + */ + table = ""; + + /** + * The privileges given to that role + * + * @generated from field: repeated string privilege_type = 4; + */ + privilegeType: string[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConnectionRolePrivilege"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "grantee", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "privilege_type", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionRolePrivilege { + return new ConnectionRolePrivilege().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionRolePrivilege { + return new ConnectionRolePrivilege().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConnectionRolePrivilege { + return new ConnectionRolePrivilege().fromJsonString(jsonString, options); + } + + static equals(a: ConnectionRolePrivilege | PlainMessage | undefined, b: ConnectionRolePrivilege | PlainMessage | undefined): boolean { + return proto3.util.equals(ConnectionRolePrivilege, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.Connection + */ +export class Connection extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: mgmt.v1alpha1.ConnectionConfig connection_config = 3; + */ + connectionConfig?: ConnectionConfig; + + /** + * @generated from field: string created_by_user_id = 4; + */ + createdByUserId = ""; + + /** + * @generated from field: google.protobuf.Timestamp created_at = 5; + */ + createdAt?: Timestamp; + + /** + * @generated from field: string updated_by_user_id = 6; + */ + updatedByUserId = ""; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 7; + */ + updatedAt?: Timestamp; + + /** + * @generated from field: string account_id = 8; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.Connection"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "connection_config", kind: "message", T: ConnectionConfig }, + { no: 4, name: "created_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "created_at", kind: "message", T: Timestamp }, + { no: 6, name: "updated_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 7, name: "updated_at", kind: "message", T: Timestamp }, + { no: 8, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): Connection { + return new Connection().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): Connection { + return new Connection().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): Connection { + return new Connection().fromJsonString(jsonString, options); + } + + static equals(a: Connection | PlainMessage | undefined, b: Connection | PlainMessage | undefined): boolean { + return proto3.util.equals(Connection, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConnectionConfig + */ +export class ConnectionConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.ConnectionConfig.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.PostgresConnectionConfig pg_config = 1; + */ + value: PostgresConnectionConfig; + case: "pgConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.AwsS3ConnectionConfig aws_s3_config = 2; + */ + value: AwsS3ConnectionConfig; + case: "awsS3Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlConnectionConfig mysql_config = 3; + */ + value: MysqlConnectionConfig; + case: "mysqlConfig"; + } | { + /** + * Configures a connection to a directory available on the local file system + * + * @generated from field: mgmt.v1alpha1.LocalDirectoryConnectionConfig local_dir_config = 4; + */ + value: LocalDirectoryConnectionConfig; + case: "localDirConfig"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConnectionConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "pg_config", kind: "message", T: PostgresConnectionConfig, oneof: "config" }, + { no: 2, name: "aws_s3_config", kind: "message", T: AwsS3ConnectionConfig, oneof: "config" }, + { no: 3, name: "mysql_config", kind: "message", T: MysqlConnectionConfig, oneof: "config" }, + { no: 4, name: "local_dir_config", kind: "message", T: LocalDirectoryConnectionConfig, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConnectionConfig { + return new ConnectionConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConnectionConfig { + return new ConnectionConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConnectionConfig { + return new ConnectionConfig().fromJsonString(jsonString, options); + } + + static equals(a: ConnectionConfig | PlainMessage | undefined, b: ConnectionConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(ConnectionConfig, a, b); + } +} + +/** + * Configures a connection to a directory available on the local file system + * + * @generated from message mgmt.v1alpha1.LocalDirectoryConnectionConfig + */ +export class LocalDirectoryConnectionConfig extends Message { + /** + * The absolute path to a directory that is available on the local file system to the API and Worker nodes + * + * @generated from field: string path = 1; + */ + path = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.LocalDirectoryConnectionConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "path", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): LocalDirectoryConnectionConfig { + return new LocalDirectoryConnectionConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): LocalDirectoryConnectionConfig { + return new LocalDirectoryConnectionConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): LocalDirectoryConnectionConfig { + return new LocalDirectoryConnectionConfig().fromJsonString(jsonString, options); + } + + static equals(a: LocalDirectoryConnectionConfig | PlainMessage | undefined, b: LocalDirectoryConnectionConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(LocalDirectoryConnectionConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresConnectionConfig + */ +export class PostgresConnectionConfig extends Message { + /** + * May provide either a raw string url, or a structured version + * + * @generated from oneof mgmt.v1alpha1.PostgresConnectionConfig.connection_config + */ + connectionConfig: { + /** + * @generated from field: string url = 1; + */ + value: string; + case: "url"; + } | { + /** + * @generated from field: mgmt.v1alpha1.PostgresConnection connection = 2; + */ + value: PostgresConnection; + case: "connection"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + /** + * Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet + * + * @generated from field: mgmt.v1alpha1.SSHTunnel tunnel = 3; + */ + tunnel?: SSHTunnel; + + /** + * @generated from field: mgmt.v1alpha1.SqlConnectionOptions connection_options = 4; + */ + connectionOptions?: SqlConnectionOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresConnectionConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "connection_config" }, + { no: 2, name: "connection", kind: "message", T: PostgresConnection, oneof: "connection_config" }, + { no: 3, name: "tunnel", kind: "message", T: SSHTunnel }, + { no: 4, name: "connection_options", kind: "message", T: SqlConnectionOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresConnectionConfig { + return new PostgresConnectionConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresConnectionConfig { + return new PostgresConnectionConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresConnectionConfig { + return new PostgresConnectionConfig().fromJsonString(jsonString, options); + } + + static equals(a: PostgresConnectionConfig | PlainMessage | undefined, b: PostgresConnectionConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresConnectionConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SqlConnectionOptions + */ +export class SqlConnectionOptions extends Message { + /** + * @generated from field: optional int32 max_connection_limit = 1; + */ + maxConnectionLimit?: number; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SqlConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "max_connection_limit", kind: "scalar", T: 5 /* ScalarType.INT32 */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SqlConnectionOptions { + return new SqlConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SqlConnectionOptions { + return new SqlConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SqlConnectionOptions { + return new SqlConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: SqlConnectionOptions | PlainMessage | undefined, b: SqlConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(SqlConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SSHTunnel + */ +export class SSHTunnel extends Message { + /** + * The host of the SSH server + * + * @generated from field: string host = 1; + */ + host = ""; + + /** + * The port of the SSH server, typically 22 + * + * @generated from field: int32 port = 2; + */ + port = 0; + + /** + * The user that will be used to authenticate + * + * @generated from field: string user = 3; + */ + user = ""; + + /** + * Optionally provide the public key of the known host of the SSH tunnel that we are connecting to. + * If this is not provided, the server will blindly connect to the host with the given credentials. + * This is not recommended for production use! + * + * @generated from field: optional string known_host_public_key = 4; + */ + knownHostPublicKey?: string; + + /** + * Provide the authentication required to successfully connect to the SSH server for tunneling + * + * @generated from field: mgmt.v1alpha1.SSHAuthentication authentication = 5; + */ + authentication?: SSHAuthentication; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SSHTunnel"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, + { no: 3, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "known_host_public_key", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 5, name: "authentication", kind: "message", T: SSHAuthentication }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SSHTunnel { + return new SSHTunnel().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SSHTunnel { + return new SSHTunnel().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SSHTunnel { + return new SSHTunnel().fromJsonString(jsonString, options); + } + + static equals(a: SSHTunnel | PlainMessage | undefined, b: SSHTunnel | PlainMessage | undefined): boolean { + return proto3.util.equals(SSHTunnel, a, b); + } +} + +/** + * SSH Authentication + * + * @generated from message mgmt.v1alpha1.SSHAuthentication + */ +export class SSHAuthentication extends Message { + /** + * @generated from oneof mgmt.v1alpha1.SSHAuthentication.auth_config + */ + authConfig: { + /** + * @generated from field: mgmt.v1alpha1.SSHPassphrase passphrase = 1; + */ + value: SSHPassphrase; + case: "passphrase"; + } | { + /** + * @generated from field: mgmt.v1alpha1.SSHPrivateKey private_key = 2; + */ + value: SSHPrivateKey; + case: "privateKey"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SSHAuthentication"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "passphrase", kind: "message", T: SSHPassphrase, oneof: "auth_config" }, + { no: 2, name: "private_key", kind: "message", T: SSHPrivateKey, oneof: "auth_config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SSHAuthentication { + return new SSHAuthentication().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SSHAuthentication { + return new SSHAuthentication().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SSHAuthentication { + return new SSHAuthentication().fromJsonString(jsonString, options); + } + + static equals(a: SSHAuthentication | PlainMessage | undefined, b: SSHAuthentication | PlainMessage | undefined): boolean { + return proto3.util.equals(SSHAuthentication, a, b); + } +} + +/** + * Contains the configuration needed to retrieve the SSH passphrase for the tunnel + * + * @generated from message mgmt.v1alpha1.SSHPassphrase + */ +export class SSHPassphrase extends Message { + /** + * The password to be used to connect to the SSH server + * + * eventually we can expand this to allow pulling from other sources. + * + * @generated from field: string value = 1; + */ + value = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SSHPassphrase"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "value", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SSHPassphrase { + return new SSHPassphrase().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SSHPassphrase { + return new SSHPassphrase().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SSHPassphrase { + return new SSHPassphrase().fromJsonString(jsonString, options); + } + + static equals(a: SSHPassphrase | PlainMessage | undefined, b: SSHPassphrase | PlainMessage | undefined): boolean { + return proto3.util.equals(SSHPassphrase, a, b); + } +} + +/** + * Contains the configuration needed to retrieve the SSH private key for the tunnel + * + * @generated from message mgmt.v1alpha1.SSHPrivateKey + */ +export class SSHPrivateKey extends Message { + /** + * The private key in PEM format. + * + * @generated from field: string value = 1; + */ + value = ""; + + /** + * If the private key is encrypted, this value should decrypt it. + * + * @generated from field: optional string passphrase = 2; + */ + passphrase?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SSHPrivateKey"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "value", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "passphrase", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SSHPrivateKey { + return new SSHPrivateKey().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SSHPrivateKey { + return new SSHPrivateKey().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SSHPrivateKey { + return new SSHPrivateKey().fromJsonString(jsonString, options); + } + + static equals(a: SSHPrivateKey | PlainMessage | undefined, b: SSHPrivateKey | PlainMessage | undefined): boolean { + return proto3.util.equals(SSHPrivateKey, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresConnection + */ +export class PostgresConnection extends Message { + /** + * @generated from field: string host = 1; + */ + host = ""; + + /** + * @generated from field: int32 port = 2; + */ + port = 0; + + /** + * @generated from field: string name = 3; + */ + name = ""; + + /** + * @generated from field: string user = 4; + */ + user = ""; + + /** + * @generated from field: string pass = 5; + */ + pass = ""; + + /** + * @generated from field: optional string ssl_mode = 6; + */ + sslMode?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresConnection"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, + { no: 3, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "pass", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 6, name: "ssl_mode", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresConnection { + return new PostgresConnection().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresConnection { + return new PostgresConnection().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresConnection { + return new PostgresConnection().fromJsonString(jsonString, options); + } + + static equals(a: PostgresConnection | PlainMessage | undefined, b: PostgresConnection | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresConnection, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlConnection + */ +export class MysqlConnection extends Message { + /** + * @generated from field: string user = 1; + */ + user = ""; + + /** + * @generated from field: string pass = 2; + */ + pass = ""; + + /** + * @generated from field: string protocol = 3; + */ + protocol = ""; + + /** + * @generated from field: string host = 4; + */ + host = ""; + + /** + * @generated from field: int32 port = 5; + */ + port = 0; + + /** + * @generated from field: string name = 6; + */ + name = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlConnection"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "user", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "pass", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "protocol", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "host", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "port", kind: "scalar", T: 5 /* ScalarType.INT32 */ }, + { no: 6, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlConnection { + return new MysqlConnection().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlConnection { + return new MysqlConnection().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlConnection { + return new MysqlConnection().fromJsonString(jsonString, options); + } + + static equals(a: MysqlConnection | PlainMessage | undefined, b: MysqlConnection | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlConnection, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlConnectionConfig + */ +export class MysqlConnectionConfig extends Message { + /** + * May provide either a raw string url, or a structured version + * + * @generated from oneof mgmt.v1alpha1.MysqlConnectionConfig.connection_config + */ + connectionConfig: { + /** + * @generated from field: string url = 1; + */ + value: string; + case: "url"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlConnection connection = 2; + */ + value: MysqlConnection; + case: "connection"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + /** + * Provide tunnel configuration that can be used to access a postgres connection that is not publicly accessible to the internet + * + * @generated from field: mgmt.v1alpha1.SSHTunnel tunnel = 3; + */ + tunnel?: SSHTunnel; + + /** + * @generated from field: mgmt.v1alpha1.SqlConnectionOptions connection_options = 4; + */ + connectionOptions?: SqlConnectionOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlConnectionConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "connection_config" }, + { no: 2, name: "connection", kind: "message", T: MysqlConnection, oneof: "connection_config" }, + { no: 3, name: "tunnel", kind: "message", T: SSHTunnel }, + { no: 4, name: "connection_options", kind: "message", T: SqlConnectionOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlConnectionConfig { + return new MysqlConnectionConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlConnectionConfig { + return new MysqlConnectionConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlConnectionConfig { + return new MysqlConnectionConfig().fromJsonString(jsonString, options); + } + + static equals(a: MysqlConnectionConfig | PlainMessage | undefined, b: MysqlConnectionConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlConnectionConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AwsS3ConnectionConfig + */ +export class AwsS3ConnectionConfig extends Message { + /** + * @generated from field: string bucket_arn = 1 [deprecated = true]; + * @deprecated + */ + bucketArn = ""; + + /** + * @generated from field: optional string path_prefix = 2; + */ + pathPrefix?: string; + + /** + * @generated from field: optional mgmt.v1alpha1.AwsS3Credentials credentials = 3; + */ + credentials?: AwsS3Credentials; + + /** + * @generated from field: optional string region = 4; + */ + region?: string; + + /** + * @generated from field: optional string endpoint = 5; + */ + endpoint?: string; + + /** + * @generated from field: string bucket = 6; + */ + bucket = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3ConnectionConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "bucket_arn", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "path_prefix", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 3, name: "credentials", kind: "message", T: AwsS3Credentials, opt: true }, + { no: 4, name: "region", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 5, name: "endpoint", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 6, name: "bucket", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3ConnectionConfig { + return new AwsS3ConnectionConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3ConnectionConfig { + return new AwsS3ConnectionConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3ConnectionConfig { + return new AwsS3ConnectionConfig().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3ConnectionConfig | PlainMessage | undefined, b: AwsS3ConnectionConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3ConnectionConfig, a, b); + } +} + +/** + * S3 Credentials that are used by the worker process. + * Note: this may be optionally provided if the worker that is being hosted has environment credentials to the S3 bucket instead. + * + * @generated from message mgmt.v1alpha1.AwsS3Credentials + */ +export class AwsS3Credentials extends Message { + /** + * @generated from field: optional string profile = 1; + */ + profile?: string; + + /** + * @generated from field: optional string access_key_id = 2; + */ + accessKeyId?: string; + + /** + * @generated from field: optional string secret_access_key = 3; + */ + secretAccessKey?: string; + + /** + * @generated from field: optional string session_token = 4; + */ + sessionToken?: string; + + /** + * @generated from field: optional bool from_ec2_role = 5; + */ + fromEc2Role?: boolean; + + /** + * @generated from field: optional string role_arn = 6; + */ + roleArn?: string; + + /** + * @generated from field: optional string role_external_id = 7; + */ + roleExternalId?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3Credentials"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "profile", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 2, name: "access_key_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 3, name: "secret_access_key", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 4, name: "session_token", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 5, name: "from_ec2_role", kind: "scalar", T: 8 /* ScalarType.BOOL */, opt: true }, + { no: 6, name: "role_arn", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 7, name: "role_external_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3Credentials { + return new AwsS3Credentials().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3Credentials { + return new AwsS3Credentials().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3Credentials { + return new AwsS3Credentials().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3Credentials | PlainMessage | undefined, b: AwsS3Credentials | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3Credentials, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsConnectionNameAvailableRequest + */ +export class IsConnectionNameAvailableRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string connection_name = 2; + */ + connectionName = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsConnectionNameAvailableRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "connection_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsConnectionNameAvailableRequest { + return new IsConnectionNameAvailableRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsConnectionNameAvailableRequest { + return new IsConnectionNameAvailableRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsConnectionNameAvailableRequest { + return new IsConnectionNameAvailableRequest().fromJsonString(jsonString, options); + } + + static equals(a: IsConnectionNameAvailableRequest | PlainMessage | undefined, b: IsConnectionNameAvailableRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(IsConnectionNameAvailableRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsConnectionNameAvailableResponse + */ +export class IsConnectionNameAvailableResponse extends Message { + /** + * @generated from field: bool is_available = 1; + */ + isAvailable = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsConnectionNameAvailableResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsConnectionNameAvailableResponse { + return new IsConnectionNameAvailableResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsConnectionNameAvailableResponse { + return new IsConnectionNameAvailableResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsConnectionNameAvailableResponse { + return new IsConnectionNameAvailableResponse().fromJsonString(jsonString, options); + } + + static equals(a: IsConnectionNameAvailableResponse | PlainMessage | undefined, b: IsConnectionNameAvailableResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(IsConnectionNameAvailableResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckSqlQueryRequest + */ +export class CheckSqlQueryRequest extends Message { + /** + * The connection id that the query will be checked against + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * The full query that will be run through a PREPARE statement + * + * @generated from field: string query = 2; + */ + query = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckSqlQueryRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "query", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckSqlQueryRequest { + return new CheckSqlQueryRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckSqlQueryRequest { + return new CheckSqlQueryRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckSqlQueryRequest { + return new CheckSqlQueryRequest().fromJsonString(jsonString, options); + } + + static equals(a: CheckSqlQueryRequest | PlainMessage | undefined, b: CheckSqlQueryRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckSqlQueryRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CheckSqlQueryResponse + */ +export class CheckSqlQueryResponse extends Message { + /** + * The query is run through PREPARE. Returns valid if it correctly compiled + * + * @generated from field: bool is_valid = 1; + */ + isValid = false; + + /** + * The error message returned by the sql client if the prepare did not return successfully + * + * @generated from field: optional string erorr_message = 2; + */ + erorrMessage?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CheckSqlQueryResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "erorr_message", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CheckSqlQueryResponse { + return new CheckSqlQueryResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CheckSqlQueryResponse { + return new CheckSqlQueryResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CheckSqlQueryResponse { + return new CheckSqlQueryResponse().fromJsonString(jsonString, options); + } + + static equals(a: CheckSqlQueryResponse | PlainMessage | undefined, b: CheckSqlQueryResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CheckSqlQueryResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts new file mode 100644 index 0000000000..326c87142d --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_connect.ts @@ -0,0 +1,265 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/job.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { CancelJobRunRequest, CancelJobRunResponse, CreateJobDestinationConnectionsRequest, CreateJobDestinationConnectionsResponse, CreateJobRequest, CreateJobResponse, CreateJobRunRequest, CreateJobRunResponse, DeleteJobDestinationConnectionRequest, DeleteJobDestinationConnectionResponse, DeleteJobRequest, DeleteJobResponse, DeleteJobRunRequest, DeleteJobRunResponse, GetJobNextRunsRequest, GetJobNextRunsResponse, GetJobRecentRunsRequest, GetJobRecentRunsResponse, GetJobRequest, GetJobResponse, GetJobRunEventsRequest, GetJobRunEventsResponse, GetJobRunLogsStreamRequest, GetJobRunLogsStreamResponse, GetJobRunRequest, GetJobRunResponse, GetJobRunsRequest, GetJobRunsResponse, GetJobsRequest, GetJobsResponse, GetJobStatusesRequest, GetJobStatusesResponse, GetJobStatusRequest, GetJobStatusResponse, IsJobNameAvailableRequest, IsJobNameAvailableResponse, PauseJobRequest, PauseJobResponse, SetJobSourceSqlConnectionSubsetsRequest, SetJobSourceSqlConnectionSubsetsResponse, SetJobSyncOptionsRequest, SetJobSyncOptionsResponse, SetJobWorkflowOptionsRequest, SetJobWorkflowOptionsResponse, TerminateJobRunRequest, TerminateJobRunResponse, UpdateJobDestinationConnectionRequest, UpdateJobDestinationConnectionResponse, UpdateJobScheduleRequest, UpdateJobScheduleResponse, UpdateJobSourceConnectionRequest, UpdateJobSourceConnectionResponse } from "./job_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * @generated from service mgmt.v1alpha1.JobService + */ +export const JobService = { + typeName: "mgmt.v1alpha1.JobService", + methods: { + /** + * @generated from rpc mgmt.v1alpha1.JobService.GetJobs + */ + getJobs: { + name: "GetJobs", + I: GetJobsRequest, + O: GetJobsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.GetJob + */ + getJob: { + name: "GetJob", + I: GetJobRequest, + O: GetJobResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.CreateJob + */ + createJob: { + name: "CreateJob", + I: CreateJobRequest, + O: CreateJobResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.DeleteJob + */ + deleteJob: { + name: "DeleteJob", + I: DeleteJobRequest, + O: DeleteJobResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.IsJobNameAvailable + */ + isJobNameAvailable: { + name: "IsJobNameAvailable", + I: IsJobNameAvailableRequest, + O: IsJobNameAvailableResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobSchedule + */ + updateJobSchedule: { + name: "UpdateJobSchedule", + I: UpdateJobScheduleRequest, + O: UpdateJobScheduleResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobSourceConnection + */ + updateJobSourceConnection: { + name: "UpdateJobSourceConnection", + I: UpdateJobSourceConnectionRequest, + O: UpdateJobSourceConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.SetJobSourceSqlConnectionSubsets + */ + setJobSourceSqlConnectionSubsets: { + name: "SetJobSourceSqlConnectionSubsets", + I: SetJobSourceSqlConnectionSubsetsRequest, + O: SetJobSourceSqlConnectionSubsetsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.UpdateJobDestinationConnection + */ + updateJobDestinationConnection: { + name: "UpdateJobDestinationConnection", + I: UpdateJobDestinationConnectionRequest, + O: UpdateJobDestinationConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.DeleteJobDestinationConnection + */ + deleteJobDestinationConnection: { + name: "DeleteJobDestinationConnection", + I: DeleteJobDestinationConnectionRequest, + O: DeleteJobDestinationConnectionResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.CreateJobDestinationConnections + */ + createJobDestinationConnections: { + name: "CreateJobDestinationConnections", + I: CreateJobDestinationConnectionsRequest, + O: CreateJobDestinationConnectionsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.PauseJob + */ + pauseJob: { + name: "PauseJob", + I: PauseJobRequest, + O: PauseJobResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a list of recently invoked job runs baseds on the Temporal cron scheduler. This will return a list of job runs that include archived runs + * + * @generated from rpc mgmt.v1alpha1.JobService.GetJobRecentRuns + */ + getJobRecentRuns: { + name: "GetJobRecentRuns", + I: GetJobRecentRunsRequest, + O: GetJobRecentRunsResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a list of runs that are scheduled for execution based on the Temporal cron scheduler. + * + * @generated from rpc mgmt.v1alpha1.JobService.GetJobNextRuns + */ + getJobNextRuns: { + name: "GetJobNextRuns", + I: GetJobNextRunsRequest, + O: GetJobNextRunsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.GetJobStatus + */ + getJobStatus: { + name: "GetJobStatus", + I: GetJobStatusRequest, + O: GetJobStatusResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.GetJobStatuses + */ + getJobStatuses: { + name: "GetJobStatuses", + I: GetJobStatusesRequest, + O: GetJobStatusesResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a list of job runs by either account or job + * + * @generated from rpc mgmt.v1alpha1.JobService.GetJobRuns + */ + getJobRuns: { + name: "GetJobRuns", + I: GetJobRunsRequest, + O: GetJobRunsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.GetJobRunEvents + */ + getJobRunEvents: { + name: "GetJobRunEvents", + I: GetJobRunEventsRequest, + O: GetJobRunEventsResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a specific job run, along with any of its pending activities + * + * @generated from rpc mgmt.v1alpha1.JobService.GetJobRun + */ + getJobRun: { + name: "GetJobRun", + I: GetJobRunRequest, + O: GetJobRunResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.DeleteJobRun + */ + deleteJobRun: { + name: "DeleteJobRun", + I: DeleteJobRunRequest, + O: DeleteJobRunResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.CreateJobRun + */ + createJobRun: { + name: "CreateJobRun", + I: CreateJobRunRequest, + O: CreateJobRunResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.CancelJobRun + */ + cancelJobRun: { + name: "CancelJobRun", + I: CancelJobRunRequest, + O: CancelJobRunResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.JobService.TerminateJobRun + */ + terminateJobRun: { + name: "TerminateJobRun", + I: TerminateJobRunRequest, + O: TerminateJobRunResponse, + kind: MethodKind.Unary, + }, + /** + * Returns a stream of logs from the worker nodes that pertain to a specific job run + * + * @generated from rpc mgmt.v1alpha1.JobService.GetJobRunLogsStream + */ + getJobRunLogsStream: { + name: "GetJobRunLogsStream", + I: GetJobRunLogsStreamRequest, + O: GetJobRunLogsStreamResponse, + kind: MethodKind.ServerStreaming, + }, + /** + * Set any job workflow options. Must provide entire object as is it will fully override the previous configuration + * + * @generated from rpc mgmt.v1alpha1.JobService.SetJobWorkflowOptions + */ + setJobWorkflowOptions: { + name: "SetJobWorkflowOptions", + I: SetJobWorkflowOptionsRequest, + O: SetJobWorkflowOptionsResponse, + kind: MethodKind.Unary, + }, + /** + * Set the job sync options. Must provide entire object as it will fully override the previous configuration + * + * @generated from rpc mgmt.v1alpha1.JobService.SetJobSyncOptions + */ + setJobSyncOptions: { + name: "SetJobSyncOptions", + I: SetJobSyncOptionsRequest, + O: SetJobSyncOptionsResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts new file mode 100644 index 0000000000..caaa9d2141 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/job_pb.ts @@ -0,0 +1,4384 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/job.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; +import { TransformerConfig, TransformerSource } from "./transformer_pb.js"; + +/** + * @generated from enum mgmt.v1alpha1.JobStatus + */ +export enum JobStatus { + /** + * @generated from enum value: JOB_STATUS_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: JOB_STATUS_ENABLED = 1; + */ + ENABLED = 1, + + /** + * @generated from enum value: JOB_STATUS_PAUSED = 3; + */ + PAUSED = 3, + + /** + * @generated from enum value: JOB_STATUS_DISABLED = 4; + */ + DISABLED = 4, +} +// Retrieve enum metadata with: proto3.getEnumType(JobStatus) +proto3.util.setEnumType(JobStatus, "mgmt.v1alpha1.JobStatus", [ + { no: 0, name: "JOB_STATUS_UNSPECIFIED" }, + { no: 1, name: "JOB_STATUS_ENABLED" }, + { no: 3, name: "JOB_STATUS_PAUSED" }, + { no: 4, name: "JOB_STATUS_DISABLED" }, +]); + +/** + * @generated from enum mgmt.v1alpha1.ActivityStatus + */ +export enum ActivityStatus { + /** + * @generated from enum value: ACTIVITY_STATUS_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: ACTIVITY_STATUS_SCHEDULED = 1; + */ + SCHEDULED = 1, + + /** + * @generated from enum value: ACTIVITY_STATUS_STARTED = 2; + */ + STARTED = 2, + + /** + * @generated from enum value: ACTIVITY_STATUS_CANCELED = 3; + */ + CANCELED = 3, + + /** + * @generated from enum value: ACTIVITY_STATUS_FAILED = 4; + */ + FAILED = 4, +} +// Retrieve enum metadata with: proto3.getEnumType(ActivityStatus) +proto3.util.setEnumType(ActivityStatus, "mgmt.v1alpha1.ActivityStatus", [ + { no: 0, name: "ACTIVITY_STATUS_UNSPECIFIED" }, + { no: 1, name: "ACTIVITY_STATUS_SCHEDULED" }, + { no: 2, name: "ACTIVITY_STATUS_STARTED" }, + { no: 3, name: "ACTIVITY_STATUS_CANCELED" }, + { no: 4, name: "ACTIVITY_STATUS_FAILED" }, +]); + +/** + * An enumeration of job run statuses. + * + * @generated from enum mgmt.v1alpha1.JobRunStatus + */ +export enum JobRunStatus { + /** + * if the job run status is unknown + * + * @generated from enum value: JOB_RUN_STATUS_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * the run is pending and has not started yet + * + * @generated from enum value: JOB_RUN_STATUS_PENDING = 1; + */ + PENDING = 1, + + /** + * the run is currently in progress + * + * @generated from enum value: JOB_RUN_STATUS_RUNNING = 2; + */ + RUNNING = 2, + + /** + * the run has successfully completed + * + * @generated from enum value: JOB_RUN_STATUS_COMPLETE = 3; + */ + COMPLETE = 3, + + /** + * the run ended with an error + * + * @generated from enum value: JOB_RUN_STATUS_ERROR = 4; + */ + ERROR = 4, + + /** + * the run was cancelled + * + * @generated from enum value: JOB_RUN_STATUS_CANCELED = 5; + */ + CANCELED = 5, + + /** + * the run was terminated + * + * @generated from enum value: JOB_RUN_STATUS_TERMINATED = 6; + */ + TERMINATED = 6, + + /** + * the run ended in failure + * + * @generated from enum value: JOB_RUN_STATUS_FAILED = 7; + */ + FAILED = 7, + + /** + * the run was ended pre-maturely due to timeout + * + * @generated from enum value: JOB_RUN_STATUS_TIMED_OUT = 8; + */ + TIMED_OUT = 8, +} +// Retrieve enum metadata with: proto3.getEnumType(JobRunStatus) +proto3.util.setEnumType(JobRunStatus, "mgmt.v1alpha1.JobRunStatus", [ + { no: 0, name: "JOB_RUN_STATUS_UNSPECIFIED" }, + { no: 1, name: "JOB_RUN_STATUS_PENDING" }, + { no: 2, name: "JOB_RUN_STATUS_RUNNING" }, + { no: 3, name: "JOB_RUN_STATUS_COMPLETE" }, + { no: 4, name: "JOB_RUN_STATUS_ERROR" }, + { no: 5, name: "JOB_RUN_STATUS_CANCELED" }, + { no: 6, name: "JOB_RUN_STATUS_TERMINATED" }, + { no: 7, name: "JOB_RUN_STATUS_FAILED" }, + { no: 8, name: "JOB_RUN_STATUS_TIMED_OUT" }, +]); + +/** + * @generated from enum mgmt.v1alpha1.LogWindow + */ +export enum LogWindow { + /** + * @generated from enum value: LOG_WINDOW_NO_TIME_UNSPECIFIED = 0; + */ + NO_TIME_UNSPECIFIED = 0, + + /** + * @generated from enum value: LOG_WINDOW_FIFTEEN_MIN = 1; + */ + FIFTEEN_MIN = 1, + + /** + * @generated from enum value: LOG_WINDOW_ONE_HOUR = 2; + */ + ONE_HOUR = 2, + + /** + * @generated from enum value: LOG_WINDOW_ONE_DAY = 3; + */ + ONE_DAY = 3, +} +// Retrieve enum metadata with: proto3.getEnumType(LogWindow) +proto3.util.setEnumType(LogWindow, "mgmt.v1alpha1.LogWindow", [ + { no: 0, name: "LOG_WINDOW_NO_TIME_UNSPECIFIED" }, + { no: 1, name: "LOG_WINDOW_FIFTEEN_MIN" }, + { no: 2, name: "LOG_WINDOW_ONE_HOUR" }, + { no: 3, name: "LOG_WINDOW_ONE_DAY" }, +]); + +/** + * @generated from enum mgmt.v1alpha1.LogLevel + */ +export enum LogLevel { + /** + * @generated from enum value: LOG_LEVEL_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: LOG_LEVEL_DEBUG = 1; + */ + DEBUG = 1, + + /** + * @generated from enum value: LOG_LEVEL_INFO = 2; + */ + INFO = 2, + + /** + * @generated from enum value: LOG_LEVEL_WARN = 3; + */ + WARN = 3, + + /** + * @generated from enum value: LOG_LEVEL_ERROR = 4; + */ + ERROR = 4, +} +// Retrieve enum metadata with: proto3.getEnumType(LogLevel) +proto3.util.setEnumType(LogLevel, "mgmt.v1alpha1.LogLevel", [ + { no: 0, name: "LOG_LEVEL_UNSPECIFIED" }, + { no: 1, name: "LOG_LEVEL_DEBUG" }, + { no: 2, name: "LOG_LEVEL_INFO" }, + { no: 3, name: "LOG_LEVEL_WARN" }, + { no: 4, name: "LOG_LEVEL_ERROR" }, +]); + +/** + * @generated from message mgmt.v1alpha1.GetJobsRequest + */ +export class GetJobsRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobsRequest { + return new GetJobsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobsRequest { + return new GetJobsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobsRequest { + return new GetJobsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobsRequest | PlainMessage | undefined, b: GetJobsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobsResponse + */ +export class GetJobsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.Job jobs = 1; + */ + jobs: Job[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "jobs", kind: "message", T: Job, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobsResponse { + return new GetJobsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobsResponse { + return new GetJobsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobsResponse { + return new GetJobsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobsResponse | PlainMessage | undefined, b: GetJobsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobSource + */ +export class JobSource extends Message { + /** + * @generated from field: mgmt.v1alpha1.JobSourceOptions options = 1; + */ + options?: JobSourceOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobSource"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "options", kind: "message", T: JobSourceOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobSource { + return new JobSource().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobSource { + return new JobSource().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobSource { + return new JobSource().fromJsonString(jsonString, options); + } + + static equals(a: JobSource | PlainMessage | undefined, b: JobSource | PlainMessage | undefined): boolean { + return proto3.util.equals(JobSource, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobSourceOptions + */ +export class JobSourceOptions extends Message { + /** + * @generated from oneof mgmt.v1alpha1.JobSourceOptions.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.PostgresSourceConnectionOptions postgres = 1; + */ + value: PostgresSourceConnectionOptions; + case: "postgres"; + } | { + /** + * @generated from field: mgmt.v1alpha1.AwsS3SourceConnectionOptions aws_s3 = 2; + */ + value: AwsS3SourceConnectionOptions; + case: "awsS3"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlSourceConnectionOptions mysql = 3; + */ + value: MysqlSourceConnectionOptions; + case: "mysql"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateSourceOptions generate = 4; + */ + value: GenerateSourceOptions; + case: "generate"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobSourceOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "postgres", kind: "message", T: PostgresSourceConnectionOptions, oneof: "config" }, + { no: 2, name: "aws_s3", kind: "message", T: AwsS3SourceConnectionOptions, oneof: "config" }, + { no: 3, name: "mysql", kind: "message", T: MysqlSourceConnectionOptions, oneof: "config" }, + { no: 4, name: "generate", kind: "message", T: GenerateSourceOptions, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobSourceOptions { + return new JobSourceOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobSourceOptions { + return new JobSourceOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobSourceOptions { + return new JobSourceOptions().fromJsonString(jsonString, options); + } + + static equals(a: JobSourceOptions | PlainMessage | undefined, b: JobSourceOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(JobSourceOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobDestination + */ +export class CreateJobDestination extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 2; + */ + options?: JobDestinationOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobDestination"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "options", kind: "message", T: JobDestinationOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestination { + return new CreateJobDestination().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestination { + return new CreateJobDestination().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobDestination { + return new CreateJobDestination().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobDestination | PlainMessage | undefined, b: CreateJobDestination | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobDestination, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobDestination + */ +export class JobDestination extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 2; + */ + options?: JobDestinationOptions; + + /** + * @generated from field: string id = 3; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobDestination"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "options", kind: "message", T: JobDestinationOptions }, + { no: 3, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobDestination { + return new JobDestination().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobDestination { + return new JobDestination().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobDestination { + return new JobDestination().fromJsonString(jsonString, options); + } + + static equals(a: JobDestination | PlainMessage | undefined, b: JobDestination | PlainMessage | undefined): boolean { + return proto3.util.equals(JobDestination, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateSourceOptions + */ +export class GenerateSourceOptions extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.GenerateSourceSchemaOption schemas = 1; + */ + schemas: GenerateSourceSchemaOption[] = []; + + /** + * @generated from field: optional string fk_source_connection_id = 3; + */ + fkSourceConnectionId?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateSourceOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schemas", kind: "message", T: GenerateSourceSchemaOption, repeated: true }, + { no: 3, name: "fk_source_connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceOptions { + return new GenerateSourceOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceOptions { + return new GenerateSourceOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateSourceOptions { + return new GenerateSourceOptions().fromJsonString(jsonString, options); + } + + static equals(a: GenerateSourceOptions | PlainMessage | undefined, b: GenerateSourceOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateSourceOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateSourceSchemaOption + */ +export class GenerateSourceSchemaOption extends Message { + /** + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * @generated from field: repeated mgmt.v1alpha1.GenerateSourceTableOption tables = 2; + */ + tables: GenerateSourceTableOption[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateSourceSchemaOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "tables", kind: "message", T: GenerateSourceTableOption, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceSchemaOption { + return new GenerateSourceSchemaOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceSchemaOption { + return new GenerateSourceSchemaOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateSourceSchemaOption { + return new GenerateSourceSchemaOption().fromJsonString(jsonString, options); + } + + static equals(a: GenerateSourceSchemaOption | PlainMessage | undefined, b: GenerateSourceSchemaOption | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateSourceSchemaOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateSourceTableOption + */ +export class GenerateSourceTableOption extends Message { + /** + * @generated from field: string table = 1; + */ + table = ""; + + /** + * @generated from field: int64 row_count = 2; + */ + rowCount = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateSourceTableOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "row_count", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSourceTableOption { + return new GenerateSourceTableOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSourceTableOption { + return new GenerateSourceTableOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateSourceTableOption { + return new GenerateSourceTableOption().fromJsonString(jsonString, options); + } + + static equals(a: GenerateSourceTableOption | PlainMessage | undefined, b: GenerateSourceTableOption | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateSourceTableOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresSourceConnectionOptions + */ +export class PostgresSourceConnectionOptions extends Message { + /** + * @generated from field: bool halt_on_new_column_addition = 1; + */ + haltOnNewColumnAddition = false; + + /** + * @generated from field: repeated mgmt.v1alpha1.PostgresSourceSchemaOption schemas = 2; + */ + schemas: PostgresSourceSchemaOption[] = []; + + /** + * @generated from field: string connection_id = 3; + */ + connectionId = ""; + + /** + * @generated from field: bool subset_by_foreign_key_constraints = 4; + */ + subsetByForeignKeyConstraints = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresSourceConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "halt_on_new_column_addition", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "schemas", kind: "message", T: PostgresSourceSchemaOption, repeated: true }, + { no: 3, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceConnectionOptions { + return new PostgresSourceConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceConnectionOptions { + return new PostgresSourceConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresSourceConnectionOptions { + return new PostgresSourceConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: PostgresSourceConnectionOptions | PlainMessage | undefined, b: PostgresSourceConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresSourceConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresSourceSchemaOption + */ +export class PostgresSourceSchemaOption extends Message { + /** + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * @generated from field: repeated mgmt.v1alpha1.PostgresSourceTableOption tables = 2; + */ + tables: PostgresSourceTableOption[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresSourceSchemaOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "tables", kind: "message", T: PostgresSourceTableOption, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceSchemaOption { + return new PostgresSourceSchemaOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceSchemaOption { + return new PostgresSourceSchemaOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresSourceSchemaOption { + return new PostgresSourceSchemaOption().fromJsonString(jsonString, options); + } + + static equals(a: PostgresSourceSchemaOption | PlainMessage | undefined, b: PostgresSourceSchemaOption | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresSourceSchemaOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresSourceTableOption + */ +export class PostgresSourceTableOption extends Message { + /** + * @generated from field: string table = 1; + */ + table = ""; + + /** + * @generated from field: optional string where_clause = 2; + */ + whereClause?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresSourceTableOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "where_clause", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceTableOption { + return new PostgresSourceTableOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceTableOption { + return new PostgresSourceTableOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresSourceTableOption { + return new PostgresSourceTableOption().fromJsonString(jsonString, options); + } + + static equals(a: PostgresSourceTableOption | PlainMessage | undefined, b: PostgresSourceTableOption | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresSourceTableOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlSourceConnectionOptions + */ +export class MysqlSourceConnectionOptions extends Message { + /** + * @generated from field: bool halt_on_new_column_addition = 1; + */ + haltOnNewColumnAddition = false; + + /** + * @generated from field: repeated mgmt.v1alpha1.MysqlSourceSchemaOption schemas = 2; + */ + schemas: MysqlSourceSchemaOption[] = []; + + /** + * @generated from field: string connection_id = 3; + */ + connectionId = ""; + + /** + * @generated from field: bool subset_by_foreign_key_constraints = 4; + */ + subsetByForeignKeyConstraints = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlSourceConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "halt_on_new_column_addition", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "schemas", kind: "message", T: MysqlSourceSchemaOption, repeated: true }, + { no: 3, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceConnectionOptions { + return new MysqlSourceConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceConnectionOptions { + return new MysqlSourceConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlSourceConnectionOptions { + return new MysqlSourceConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: MysqlSourceConnectionOptions | PlainMessage | undefined, b: MysqlSourceConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlSourceConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlSourceSchemaOption + */ +export class MysqlSourceSchemaOption extends Message { + /** + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * @generated from field: repeated mgmt.v1alpha1.MysqlSourceTableOption tables = 2; + */ + tables: MysqlSourceTableOption[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlSourceSchemaOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "tables", kind: "message", T: MysqlSourceTableOption, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceSchemaOption { + return new MysqlSourceSchemaOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceSchemaOption { + return new MysqlSourceSchemaOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlSourceSchemaOption { + return new MysqlSourceSchemaOption().fromJsonString(jsonString, options); + } + + static equals(a: MysqlSourceSchemaOption | PlainMessage | undefined, b: MysqlSourceSchemaOption | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlSourceSchemaOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlSourceTableOption + */ +export class MysqlSourceTableOption extends Message { + /** + * @generated from field: string table = 1; + */ + table = ""; + + /** + * @generated from field: optional string where_clause = 2; + */ + whereClause?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlSourceTableOption"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "where_clause", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceTableOption { + return new MysqlSourceTableOption().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceTableOption { + return new MysqlSourceTableOption().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlSourceTableOption { + return new MysqlSourceTableOption().fromJsonString(jsonString, options); + } + + static equals(a: MysqlSourceTableOption | PlainMessage | undefined, b: MysqlSourceTableOption | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlSourceTableOption, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AwsS3SourceConnectionOptions + */ +export class AwsS3SourceConnectionOptions extends Message { + /** + * @generated from field: string connection_id = 1; + */ + connectionId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3SourceConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3SourceConnectionOptions { + return new AwsS3SourceConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3SourceConnectionOptions { + return new AwsS3SourceConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3SourceConnectionOptions { + return new AwsS3SourceConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3SourceConnectionOptions | PlainMessage | undefined, b: AwsS3SourceConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3SourceConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobDestinationOptions + */ +export class JobDestinationOptions extends Message { + /** + * @generated from oneof mgmt.v1alpha1.JobDestinationOptions.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.PostgresDestinationConnectionOptions postgres_options = 1; + */ + value: PostgresDestinationConnectionOptions; + case: "postgresOptions"; + } | { + /** + * @generated from field: mgmt.v1alpha1.AwsS3DestinationConnectionOptions aws_s3_options = 2; + */ + value: AwsS3DestinationConnectionOptions; + case: "awsS3Options"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlDestinationConnectionOptions mysql_options = 3; + */ + value: MysqlDestinationConnectionOptions; + case: "mysqlOptions"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobDestinationOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "postgres_options", kind: "message", T: PostgresDestinationConnectionOptions, oneof: "config" }, + { no: 2, name: "aws_s3_options", kind: "message", T: AwsS3DestinationConnectionOptions, oneof: "config" }, + { no: 3, name: "mysql_options", kind: "message", T: MysqlDestinationConnectionOptions, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobDestinationOptions { + return new JobDestinationOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobDestinationOptions { + return new JobDestinationOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobDestinationOptions { + return new JobDestinationOptions().fromJsonString(jsonString, options); + } + + static equals(a: JobDestinationOptions | PlainMessage | undefined, b: JobDestinationOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(JobDestinationOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresDestinationConnectionOptions + */ +export class PostgresDestinationConnectionOptions extends Message { + /** + * @generated from field: mgmt.v1alpha1.PostgresTruncateTableConfig truncate_table = 1; + */ + truncateTable?: PostgresTruncateTableConfig; + + /** + * @generated from field: bool init_table_schema = 2; + */ + initTableSchema = false; + + /** + * @generated from field: mgmt.v1alpha1.PostgresOnConflictConfig on_conflict = 3; + */ + onConflict?: PostgresOnConflictConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresDestinationConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "truncate_table", kind: "message", T: PostgresTruncateTableConfig }, + { no: 2, name: "init_table_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "on_conflict", kind: "message", T: PostgresOnConflictConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresDestinationConnectionOptions { + return new PostgresDestinationConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresDestinationConnectionOptions { + return new PostgresDestinationConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresDestinationConnectionOptions { + return new PostgresDestinationConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: PostgresDestinationConnectionOptions | PlainMessage | undefined, b: PostgresDestinationConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresDestinationConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresOnConflictConfig + */ +export class PostgresOnConflictConfig extends Message { + /** + * @generated from field: bool do_nothing = 1; + */ + doNothing = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresOnConflictConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "do_nothing", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresOnConflictConfig { + return new PostgresOnConflictConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresOnConflictConfig { + return new PostgresOnConflictConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresOnConflictConfig { + return new PostgresOnConflictConfig().fromJsonString(jsonString, options); + } + + static equals(a: PostgresOnConflictConfig | PlainMessage | undefined, b: PostgresOnConflictConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresOnConflictConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresTruncateTableConfig + */ +export class PostgresTruncateTableConfig extends Message { + /** + * @generated from field: bool truncate_before_insert = 1; + */ + truncateBeforeInsert = false; + + /** + * @generated from field: bool cascade = 2; + */ + cascade = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresTruncateTableConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "cascade", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresTruncateTableConfig { + return new PostgresTruncateTableConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresTruncateTableConfig { + return new PostgresTruncateTableConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresTruncateTableConfig { + return new PostgresTruncateTableConfig().fromJsonString(jsonString, options); + } + + static equals(a: PostgresTruncateTableConfig | PlainMessage | undefined, b: PostgresTruncateTableConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresTruncateTableConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlDestinationConnectionOptions + */ +export class MysqlDestinationConnectionOptions extends Message { + /** + * @generated from field: mgmt.v1alpha1.MysqlTruncateTableConfig truncate_table = 1; + */ + truncateTable?: MysqlTruncateTableConfig; + + /** + * @generated from field: bool init_table_schema = 2; + */ + initTableSchema = false; + + /** + * @generated from field: mgmt.v1alpha1.MysqlOnConflictConfig on_conflict = 3; + */ + onConflict?: MysqlOnConflictConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlDestinationConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "truncate_table", kind: "message", T: MysqlTruncateTableConfig }, + { no: 2, name: "init_table_schema", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "on_conflict", kind: "message", T: MysqlOnConflictConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlDestinationConnectionOptions { + return new MysqlDestinationConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlDestinationConnectionOptions { + return new MysqlDestinationConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlDestinationConnectionOptions { + return new MysqlDestinationConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: MysqlDestinationConnectionOptions | PlainMessage | undefined, b: MysqlDestinationConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlDestinationConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlTruncateTableConfig + */ +export class MysqlTruncateTableConfig extends Message { + /** + * @generated from field: bool truncate_before_insert = 1; + */ + truncateBeforeInsert = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlTruncateTableConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "truncate_before_insert", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlTruncateTableConfig { + return new MysqlTruncateTableConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlTruncateTableConfig { + return new MysqlTruncateTableConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlTruncateTableConfig { + return new MysqlTruncateTableConfig().fromJsonString(jsonString, options); + } + + static equals(a: MysqlTruncateTableConfig | PlainMessage | undefined, b: MysqlTruncateTableConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlTruncateTableConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlOnConflictConfig + */ +export class MysqlOnConflictConfig extends Message { + /** + * @generated from field: bool do_nothing = 1; + */ + doNothing = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlOnConflictConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "do_nothing", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlOnConflictConfig { + return new MysqlOnConflictConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlOnConflictConfig { + return new MysqlOnConflictConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlOnConflictConfig { + return new MysqlOnConflictConfig().fromJsonString(jsonString, options); + } + + static equals(a: MysqlOnConflictConfig | PlainMessage | undefined, b: MysqlOnConflictConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlOnConflictConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AwsS3DestinationConnectionOptions + */ +export class AwsS3DestinationConnectionOptions extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AwsS3DestinationConnectionOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AwsS3DestinationConnectionOptions { + return new AwsS3DestinationConnectionOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AwsS3DestinationConnectionOptions { + return new AwsS3DestinationConnectionOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AwsS3DestinationConnectionOptions { + return new AwsS3DestinationConnectionOptions().fromJsonString(jsonString, options); + } + + static equals(a: AwsS3DestinationConnectionOptions | PlainMessage | undefined, b: AwsS3DestinationConnectionOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(AwsS3DestinationConnectionOptions, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobRequest + */ +export class CreateJobRequest extends Message { + /** + * The unique account identifier that this job will be associated with + * + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * The unique, friendly name of the job. This is unique per account + * + * @generated from field: string job_name = 2; + */ + jobName = ""; + + /** + * Optionally provide a cron schedule. Goes into effect if the job status is set to enabled + * + * @generated from field: optional string cron_schedule = 3; + */ + cronSchedule?: string; + + /** + * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 4; + */ + mappings: JobMapping[] = []; + + /** + * @generated from field: mgmt.v1alpha1.JobSource source = 5; + */ + source?: JobSource; + + /** + * @generated from field: repeated mgmt.v1alpha1.CreateJobDestination destinations = 6; + */ + destinations: CreateJobDestination[] = []; + + /** + * Initially trigger a run of this job regardless of its status or cron schedule + * + * @generated from field: bool initiate_job_run = 7; + */ + initiateJobRun = false; + + /** + * Specify timeouts and other workflow options for the underlying temporal workflow + * + * @generated from field: mgmt.v1alpha1.WorkflowOptions workflow_options = 8; + */ + workflowOptions?: WorkflowOptions; + + /** + * Specify timeout and retry options for data synchronization activities + * Data sync activities are any piece of work that involves actually synchronizing data from a source to a destination + * For the data sync and generate jobs, this will be applied per table + * + * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 9; + */ + syncOptions?: ActivityOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "job_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 4, name: "mappings", kind: "message", T: JobMapping, repeated: true }, + { no: 5, name: "source", kind: "message", T: JobSource }, + { no: 6, name: "destinations", kind: "message", T: CreateJobDestination, repeated: true }, + { no: 7, name: "initiate_job_run", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 8, name: "workflow_options", kind: "message", T: WorkflowOptions }, + { no: 9, name: "sync_options", kind: "message", T: ActivityOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRequest { + return new CreateJobRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRequest { + return new CreateJobRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobRequest { + return new CreateJobRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobRequest | PlainMessage | undefined, b: CreateJobRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobRequest, a, b); + } +} + +/** + * Config that contains various timeouts that are configured in the underlying temporal workflow + * More options will come in the future as needed + * + * @generated from message mgmt.v1alpha1.WorkflowOptions + */ +export class WorkflowOptions extends Message { + /** + * The timeout for a single workflow run. + * Measured in seconds + * + * @generated from field: optional int64 run_timeout = 8; + */ + runTimeout?: bigint; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.WorkflowOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 8, name: "run_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): WorkflowOptions { + return new WorkflowOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): WorkflowOptions { + return new WorkflowOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): WorkflowOptions { + return new WorkflowOptions().fromJsonString(jsonString, options); + } + + static equals(a: WorkflowOptions | PlainMessage | undefined, b: WorkflowOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(WorkflowOptions, a, b); + } +} + +/** + * Config that contains various timeouts that are configured in the underlying temporal workflow(s) and activities + * + * @generated from message mgmt.v1alpha1.ActivityOptions + */ +export class ActivityOptions extends Message { + /** + * Total time that a workflow is willing to wait for an activity to complete, including retries. + * Measured in seconds + * + * @generated from field: optional int64 schedule_to_close_timeout = 1; + */ + scheduleToCloseTimeout?: bigint; + + /** + * Max time of a single Temporal Activity execution attempt. + * This timeout should be as short as the longest psosible execution of any activity (e.g. table sync). + * Important to know that this is per retry attempt. Defaults to the schedule to close timeout if not provided. + * Measured in seconds + * + * @generated from field: optional int64 start_to_close_timeout = 2; + */ + startToCloseTimeout?: bigint; + + /** + * Optionally define a retry policy for the activity + * If max attempts is not set, the activity will retry indefinitely until the start to close timeout lapses + * + * @generated from field: mgmt.v1alpha1.RetryPolicy retry_policy = 3; + */ + retryPolicy?: RetryPolicy; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ActivityOptions"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schedule_to_close_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, + { no: 2, name: "start_to_close_timeout", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, + { no: 3, name: "retry_policy", kind: "message", T: RetryPolicy }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ActivityOptions { + return new ActivityOptions().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ActivityOptions { + return new ActivityOptions().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ActivityOptions { + return new ActivityOptions().fromJsonString(jsonString, options); + } + + static equals(a: ActivityOptions | PlainMessage | undefined, b: ActivityOptions | PlainMessage | undefined): boolean { + return proto3.util.equals(ActivityOptions, a, b); + } +} + +/** + * Defines the retry policy for an activity + * + * @generated from message mgmt.v1alpha1.RetryPolicy + */ +export class RetryPolicy extends Message { + /** + * Maximum number of attempts. When exceeded the retries stop even if not expired yet. + * If not set or set to 0, it means unlimited, and rely on activity ScheduleToCloseTimeout to stop. + * + * @generated from field: optional int32 maximum_attempts = 1; + */ + maximumAttempts?: number; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RetryPolicy"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "maximum_attempts", kind: "scalar", T: 5 /* ScalarType.INT32 */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RetryPolicy { + return new RetryPolicy().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RetryPolicy { + return new RetryPolicy().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RetryPolicy { + return new RetryPolicy().fromJsonString(jsonString, options); + } + + static equals(a: RetryPolicy | PlainMessage | undefined, b: RetryPolicy | PlainMessage | undefined): boolean { + return proto3.util.equals(RetryPolicy, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobResponse + */ +export class CreateJobResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobResponse { + return new CreateJobResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobResponse { + return new CreateJobResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobResponse { + return new CreateJobResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobResponse | PlainMessage | undefined, b: CreateJobResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobMappingTransformer + */ +export class JobMappingTransformer extends Message { + /** + * @generated from field: mgmt.v1alpha1.TransformerSource source = 1; + */ + source = TransformerSource.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerConfig config = 3; + */ + config?: TransformerConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobMappingTransformer"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, + { no: 3, name: "config", kind: "message", T: TransformerConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobMappingTransformer { + return new JobMappingTransformer().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobMappingTransformer { + return new JobMappingTransformer().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobMappingTransformer { + return new JobMappingTransformer().fromJsonString(jsonString, options); + } + + static equals(a: JobMappingTransformer | PlainMessage | undefined, b: JobMappingTransformer | PlainMessage | undefined): boolean { + return proto3.util.equals(JobMappingTransformer, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobMapping + */ +export class JobMapping extends Message { + /** + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * @generated from field: string table = 2; + */ + table = ""; + + /** + * @generated from field: string column = 3; + */ + column = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobMappingTransformer transformer = 5; + */ + transformer?: JobMappingTransformer; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobMapping"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "column", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "transformer", kind: "message", T: JobMappingTransformer }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobMapping { + return new JobMapping().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobMapping { + return new JobMapping().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobMapping { + return new JobMapping().fromJsonString(jsonString, options); + } + + static equals(a: JobMapping | PlainMessage | undefined, b: JobMapping | PlainMessage | undefined): boolean { + return proto3.util.equals(JobMapping, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRequest + */ +export class GetJobRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRequest { + return new GetJobRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRequest { + return new GetJobRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRequest { + return new GetJobRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRequest | PlainMessage | undefined, b: GetJobRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobResponse + */ +export class GetJobResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobResponse { + return new GetJobResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobResponse { + return new GetJobResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobResponse { + return new GetJobResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobResponse | PlainMessage | undefined, b: GetJobResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobScheduleRequest + */ +export class UpdateJobScheduleRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: optional string cron_schedule = 2; + */ + cronSchedule?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobScheduleRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobScheduleRequest { + return new UpdateJobScheduleRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobScheduleRequest { + return new UpdateJobScheduleRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobScheduleRequest { + return new UpdateJobScheduleRequest().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobScheduleRequest | PlainMessage | undefined, b: UpdateJobScheduleRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobScheduleRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobScheduleResponse + */ +export class UpdateJobScheduleResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobScheduleResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobScheduleResponse { + return new UpdateJobScheduleResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobScheduleResponse { + return new UpdateJobScheduleResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobScheduleResponse { + return new UpdateJobScheduleResponse().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobScheduleResponse | PlainMessage | undefined, b: UpdateJobScheduleResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobScheduleResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PauseJobRequest + */ +export class PauseJobRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: bool pause = 2; + */ + pause = false; + + /** + * @generated from field: optional string note = 3; + */ + note?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PauseJobRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "pause", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "note", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PauseJobRequest { + return new PauseJobRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PauseJobRequest { + return new PauseJobRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PauseJobRequest { + return new PauseJobRequest().fromJsonString(jsonString, options); + } + + static equals(a: PauseJobRequest | PlainMessage | undefined, b: PauseJobRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(PauseJobRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PauseJobResponse + */ +export class PauseJobResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PauseJobResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PauseJobResponse { + return new PauseJobResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PauseJobResponse { + return new PauseJobResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PauseJobResponse { + return new PauseJobResponse().fromJsonString(jsonString, options); + } + + static equals(a: PauseJobResponse | PlainMessage | undefined, b: PauseJobResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(PauseJobResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobSourceConnectionRequest + */ +export class UpdateJobSourceConnectionRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobSource source = 2; + */ + source?: JobSource; + + /** + * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 3; + */ + mappings: JobMapping[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobSourceConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "source", kind: "message", T: JobSource }, + { no: 3, name: "mappings", kind: "message", T: JobMapping, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobSourceConnectionRequest { + return new UpdateJobSourceConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobSourceConnectionRequest { + return new UpdateJobSourceConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobSourceConnectionRequest { + return new UpdateJobSourceConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobSourceConnectionRequest | PlainMessage | undefined, b: UpdateJobSourceConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobSourceConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobSourceConnectionResponse + */ +export class UpdateJobSourceConnectionResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobSourceConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobSourceConnectionResponse { + return new UpdateJobSourceConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobSourceConnectionResponse { + return new UpdateJobSourceConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobSourceConnectionResponse { + return new UpdateJobSourceConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobSourceConnectionResponse | PlainMessage | undefined, b: UpdateJobSourceConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobSourceConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PostgresSourceSchemaSubset + */ +export class PostgresSourceSchemaSubset extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.PostgresSourceSchemaOption postgres_schemas = 1; + */ + postgresSchemas: PostgresSourceSchemaOption[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PostgresSourceSchemaSubset"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "postgres_schemas", kind: "message", T: PostgresSourceSchemaOption, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PostgresSourceSchemaSubset { + return new PostgresSourceSchemaSubset().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PostgresSourceSchemaSubset { + return new PostgresSourceSchemaSubset().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PostgresSourceSchemaSubset { + return new PostgresSourceSchemaSubset().fromJsonString(jsonString, options); + } + + static equals(a: PostgresSourceSchemaSubset | PlainMessage | undefined, b: PostgresSourceSchemaSubset | PlainMessage | undefined): boolean { + return proto3.util.equals(PostgresSourceSchemaSubset, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.MysqlSourceSchemaSubset + */ +export class MysqlSourceSchemaSubset extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.MysqlSourceSchemaOption mysql_schemas = 1; + */ + mysqlSchemas: MysqlSourceSchemaOption[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.MysqlSourceSchemaSubset"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "mysql_schemas", kind: "message", T: MysqlSourceSchemaOption, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): MysqlSourceSchemaSubset { + return new MysqlSourceSchemaSubset().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): MysqlSourceSchemaSubset { + return new MysqlSourceSchemaSubset().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): MysqlSourceSchemaSubset { + return new MysqlSourceSchemaSubset().fromJsonString(jsonString, options); + } + + static equals(a: MysqlSourceSchemaSubset | PlainMessage | undefined, b: MysqlSourceSchemaSubset | PlainMessage | undefined): boolean { + return proto3.util.equals(MysqlSourceSchemaSubset, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobSourceSqlSubetSchemas + */ +export class JobSourceSqlSubetSchemas extends Message { + /** + * @generated from oneof mgmt.v1alpha1.JobSourceSqlSubetSchemas.schemas + */ + schemas: { + /** + * @generated from field: mgmt.v1alpha1.PostgresSourceSchemaSubset postgres_subset = 2; + */ + value: PostgresSourceSchemaSubset; + case: "postgresSubset"; + } | { + /** + * @generated from field: mgmt.v1alpha1.MysqlSourceSchemaSubset mysql_subset = 3; + */ + value: MysqlSourceSchemaSubset; + case: "mysqlSubset"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobSourceSqlSubetSchemas"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 2, name: "postgres_subset", kind: "message", T: PostgresSourceSchemaSubset, oneof: "schemas" }, + { no: 3, name: "mysql_subset", kind: "message", T: MysqlSourceSchemaSubset, oneof: "schemas" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobSourceSqlSubetSchemas { + return new JobSourceSqlSubetSchemas().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobSourceSqlSubetSchemas { + return new JobSourceSqlSubetSchemas().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobSourceSqlSubetSchemas { + return new JobSourceSqlSubetSchemas().fromJsonString(jsonString, options); + } + + static equals(a: JobSourceSqlSubetSchemas | PlainMessage | undefined, b: JobSourceSqlSubetSchemas | PlainMessage | undefined): boolean { + return proto3.util.equals(JobSourceSqlSubetSchemas, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest + */ +export class SetJobSourceSqlConnectionSubsetsRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobSourceSqlSubetSchemas schemas = 2; + */ + schemas?: JobSourceSqlSubetSchemas; + + /** + * @generated from field: bool subset_by_foreign_key_constraints = 3; + */ + subsetByForeignKeyConstraints = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "schemas", kind: "message", T: JobSourceSqlSubetSchemas }, + { no: 3, name: "subset_by_foreign_key_constraints", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { + return new SetJobSourceSqlConnectionSubsetsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { + return new SetJobSourceSqlConnectionSubsetsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobSourceSqlConnectionSubsetsRequest { + return new SetJobSourceSqlConnectionSubsetsRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetJobSourceSqlConnectionSubsetsRequest | PlainMessage | undefined, b: SetJobSourceSqlConnectionSubsetsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobSourceSqlConnectionSubsetsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse + */ +export class SetJobSourceSqlConnectionSubsetsResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobSourceSqlConnectionSubsetsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { + return new SetJobSourceSqlConnectionSubsetsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { + return new SetJobSourceSqlConnectionSubsetsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobSourceSqlConnectionSubsetsResponse { + return new SetJobSourceSqlConnectionSubsetsResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetJobSourceSqlConnectionSubsetsResponse | PlainMessage | undefined, b: SetJobSourceSqlConnectionSubsetsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobSourceSqlConnectionSubsetsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobDestinationConnectionRequest + */ +export class UpdateJobDestinationConnectionRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + /** + * @generated from field: string connection_id = 2; + */ + connectionId = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobDestinationOptions options = 3; + */ + options?: JobDestinationOptions; + + /** + * @generated from field: string destination_id = 4; + */ + destinationId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobDestinationConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "connection_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "options", kind: "message", T: JobDestinationOptions }, + { no: 4, name: "destination_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobDestinationConnectionRequest { + return new UpdateJobDestinationConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobDestinationConnectionRequest { + return new UpdateJobDestinationConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobDestinationConnectionRequest { + return new UpdateJobDestinationConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobDestinationConnectionRequest | PlainMessage | undefined, b: UpdateJobDestinationConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobDestinationConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateJobDestinationConnectionResponse + */ +export class UpdateJobDestinationConnectionResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateJobDestinationConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateJobDestinationConnectionResponse { + return new UpdateJobDestinationConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateJobDestinationConnectionResponse { + return new UpdateJobDestinationConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateJobDestinationConnectionResponse { + return new UpdateJobDestinationConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: UpdateJobDestinationConnectionResponse | PlainMessage | undefined, b: UpdateJobDestinationConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateJobDestinationConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobDestinationConnectionRequest + */ +export class DeleteJobDestinationConnectionRequest extends Message { + /** + * @generated from field: string destination_id = 1; + */ + destinationId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobDestinationConnectionRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "destination_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobDestinationConnectionRequest { + return new DeleteJobDestinationConnectionRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobDestinationConnectionRequest { + return new DeleteJobDestinationConnectionRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobDestinationConnectionRequest { + return new DeleteJobDestinationConnectionRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobDestinationConnectionRequest | PlainMessage | undefined, b: DeleteJobDestinationConnectionRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobDestinationConnectionRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobDestinationConnectionResponse + */ +export class DeleteJobDestinationConnectionResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobDestinationConnectionResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobDestinationConnectionResponse { + return new DeleteJobDestinationConnectionResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobDestinationConnectionResponse { + return new DeleteJobDestinationConnectionResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobDestinationConnectionResponse { + return new DeleteJobDestinationConnectionResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobDestinationConnectionResponse | PlainMessage | undefined, b: DeleteJobDestinationConnectionResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobDestinationConnectionResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobDestinationConnectionsRequest + */ +export class CreateJobDestinationConnectionsRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + /** + * @generated from field: repeated mgmt.v1alpha1.CreateJobDestination destinations = 2; + */ + destinations: CreateJobDestination[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobDestinationConnectionsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "destinations", kind: "message", T: CreateJobDestination, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestinationConnectionsRequest { + return new CreateJobDestinationConnectionsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestinationConnectionsRequest { + return new CreateJobDestinationConnectionsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobDestinationConnectionsRequest { + return new CreateJobDestinationConnectionsRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobDestinationConnectionsRequest | PlainMessage | undefined, b: CreateJobDestinationConnectionsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobDestinationConnectionsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobDestinationConnectionsResponse + */ +export class CreateJobDestinationConnectionsResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobDestinationConnectionsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobDestinationConnectionsResponse { + return new CreateJobDestinationConnectionsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobDestinationConnectionsResponse { + return new CreateJobDestinationConnectionsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobDestinationConnectionsResponse { + return new CreateJobDestinationConnectionsResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobDestinationConnectionsResponse | PlainMessage | undefined, b: CreateJobDestinationConnectionsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobDestinationConnectionsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobRequest + */ +export class DeleteJobRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRequest { + return new DeleteJobRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRequest { + return new DeleteJobRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobRequest { + return new DeleteJobRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobRequest | PlainMessage | undefined, b: DeleteJobRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobResponse + */ +export class DeleteJobResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobResponse { + return new DeleteJobResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobResponse { + return new DeleteJobResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobResponse { + return new DeleteJobResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobResponse | PlainMessage | undefined, b: DeleteJobResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsJobNameAvailableRequest + */ +export class IsJobNameAvailableRequest extends Message { + /** + * @generated from field: string name = 1; + */ + name = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsJobNameAvailableRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsJobNameAvailableRequest { + return new IsJobNameAvailableRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsJobNameAvailableRequest { + return new IsJobNameAvailableRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsJobNameAvailableRequest { + return new IsJobNameAvailableRequest().fromJsonString(jsonString, options); + } + + static equals(a: IsJobNameAvailableRequest | PlainMessage | undefined, b: IsJobNameAvailableRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(IsJobNameAvailableRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsJobNameAvailableResponse + */ +export class IsJobNameAvailableResponse extends Message { + /** + * @generated from field: bool is_available = 1; + */ + isAvailable = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsJobNameAvailableResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsJobNameAvailableResponse { + return new IsJobNameAvailableResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsJobNameAvailableResponse { + return new IsJobNameAvailableResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsJobNameAvailableResponse { + return new IsJobNameAvailableResponse().fromJsonString(jsonString, options); + } + + static equals(a: IsJobNameAvailableResponse | PlainMessage | undefined, b: IsJobNameAvailableResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(IsJobNameAvailableResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunsRequest + */ +export class GetJobRunsRequest extends Message { + /** + * @generated from oneof mgmt.v1alpha1.GetJobRunsRequest.id + */ + id: { + /** + * @generated from field: string job_id = 1; + */ + value: string; + case: "jobId"; + } | { + /** + * @generated from field: string account_id = 2; + */ + value: string; + case: "accountId"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "id" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunsRequest { + return new GetJobRunsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunsRequest { + return new GetJobRunsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunsRequest { + return new GetJobRunsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunsRequest | PlainMessage | undefined, b: GetJobRunsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunsResponse + */ +export class GetJobRunsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.JobRun job_runs = 1; + */ + jobRuns: JobRun[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_runs", kind: "message", T: JobRun, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunsResponse { + return new GetJobRunsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunsResponse { + return new GetJobRunsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunsResponse { + return new GetJobRunsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunsResponse | PlainMessage | undefined, b: GetJobRunsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunRequest + */ +export class GetJobRunRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunRequest { + return new GetJobRunRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunRequest { + return new GetJobRunRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunRequest { + return new GetJobRunRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunRequest | PlainMessage | undefined, b: GetJobRunRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunResponse + */ +export class GetJobRunResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.JobRun job_run = 1; + */ + jobRun?: JobRun; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run", kind: "message", T: JobRun }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunResponse { + return new GetJobRunResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunResponse { + return new GetJobRunResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunResponse { + return new GetJobRunResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunResponse | PlainMessage | undefined, b: GetJobRunResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobRunRequest + */ +export class CreateJobRunRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobRunRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRunRequest { + return new CreateJobRunRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRunRequest { + return new CreateJobRunRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobRunRequest { + return new CreateJobRunRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobRunRequest | PlainMessage | undefined, b: CreateJobRunRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobRunRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateJobRunResponse + */ +export class CreateJobRunResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateJobRunResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateJobRunResponse { + return new CreateJobRunResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateJobRunResponse { + return new CreateJobRunResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateJobRunResponse { + return new CreateJobRunResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateJobRunResponse | PlainMessage | undefined, b: CreateJobRunResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateJobRunResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CancelJobRunRequest + */ +export class CancelJobRunRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CancelJobRunRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CancelJobRunRequest { + return new CancelJobRunRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CancelJobRunRequest { + return new CancelJobRunRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CancelJobRunRequest { + return new CancelJobRunRequest().fromJsonString(jsonString, options); + } + + static equals(a: CancelJobRunRequest | PlainMessage | undefined, b: CancelJobRunRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CancelJobRunRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CancelJobRunResponse + */ +export class CancelJobRunResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CancelJobRunResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CancelJobRunResponse { + return new CancelJobRunResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CancelJobRunResponse { + return new CancelJobRunResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CancelJobRunResponse { + return new CancelJobRunResponse().fromJsonString(jsonString, options); + } + + static equals(a: CancelJobRunResponse | PlainMessage | undefined, b: CancelJobRunResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CancelJobRunResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.Job + */ +export class Job extends Message { + /** + * The unique identifier of the job + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string created_by_user_id = 2; + */ + createdByUserId = ""; + + /** + * @generated from field: google.protobuf.Timestamp created_at = 3; + */ + createdAt?: Timestamp; + + /** + * @generated from field: string updated_by_user_id = 4; + */ + updatedByUserId = ""; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 5; + */ + updatedAt?: Timestamp; + + /** + * The unique, friendly name of the job + * + * @generated from field: string name = 6; + */ + name = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobSource source = 7; + */ + source?: JobSource; + + /** + * @generated from field: repeated mgmt.v1alpha1.JobDestination destinations = 8; + */ + destinations: JobDestination[] = []; + + /** + * @generated from field: repeated mgmt.v1alpha1.JobMapping mappings = 9; + */ + mappings: JobMapping[] = []; + + /** + * @generated from field: optional string cron_schedule = 10; + */ + cronSchedule?: string; + + /** + * The account identifier that a job is associated with + * + * @generated from field: string account_id = 11; + */ + accountId = ""; + + /** + * Specify timeout and retry options for data synchronization activities + * Data sync activities are any piece of work that involves actually synchronizing data from a source to a destination + * For the data sync and generate jobs, this will be applied per table + * + * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 12; + */ + syncOptions?: ActivityOptions; + + /** + * Specify timeouts and other workflow options for the underlying temporal workflow + * + * @generated from field: mgmt.v1alpha1.WorkflowOptions workflow_options = 13; + */ + workflowOptions?: WorkflowOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.Job"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "created_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "created_at", kind: "message", T: Timestamp }, + { no: 4, name: "updated_by_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "updated_at", kind: "message", T: Timestamp }, + { no: 6, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 7, name: "source", kind: "message", T: JobSource }, + { no: 8, name: "destinations", kind: "message", T: JobDestination, repeated: true }, + { no: 9, name: "mappings", kind: "message", T: JobMapping, repeated: true }, + { no: 10, name: "cron_schedule", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + { no: 11, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 12, name: "sync_options", kind: "message", T: ActivityOptions }, + { no: 13, name: "workflow_options", kind: "message", T: WorkflowOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): Job { + return new Job().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): Job { + return new Job().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): Job { + return new Job().fromJsonString(jsonString, options); + } + + static equals(a: Job | PlainMessage | undefined, b: Job | PlainMessage | undefined): boolean { + return proto3.util.equals(Job, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRecentRun + */ +export class JobRecentRun extends Message { + /** + * @generated from field: google.protobuf.Timestamp start_time = 1; + */ + startTime?: Timestamp; + + /** + * @generated from field: string job_run_id = 2; + */ + jobRunId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRecentRun"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "start_time", kind: "message", T: Timestamp }, + { no: 2, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRecentRun { + return new JobRecentRun().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRecentRun { + return new JobRecentRun().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRecentRun { + return new JobRecentRun().fromJsonString(jsonString, options); + } + + static equals(a: JobRecentRun | PlainMessage | undefined, b: JobRecentRun | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRecentRun, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRecentRunsRequest + */ +export class GetJobRecentRunsRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRecentRunsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRecentRunsRequest { + return new GetJobRecentRunsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRecentRunsRequest { + return new GetJobRecentRunsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRecentRunsRequest { + return new GetJobRecentRunsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRecentRunsRequest | PlainMessage | undefined, b: GetJobRecentRunsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRecentRunsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRecentRunsResponse + */ +export class GetJobRecentRunsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.JobRecentRun recent_runs = 1; + */ + recentRuns: JobRecentRun[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRecentRunsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "recent_runs", kind: "message", T: JobRecentRun, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRecentRunsResponse { + return new GetJobRecentRunsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRecentRunsResponse { + return new GetJobRecentRunsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRecentRunsResponse { + return new GetJobRecentRunsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRecentRunsResponse | PlainMessage | undefined, b: GetJobRecentRunsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRecentRunsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobNextRuns + */ +export class JobNextRuns extends Message { + /** + * @generated from field: repeated google.protobuf.Timestamp next_run_times = 1; + */ + nextRunTimes: Timestamp[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobNextRuns"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "next_run_times", kind: "message", T: Timestamp, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobNextRuns { + return new JobNextRuns().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobNextRuns { + return new JobNextRuns().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobNextRuns { + return new JobNextRuns().fromJsonString(jsonString, options); + } + + static equals(a: JobNextRuns | PlainMessage | undefined, b: JobNextRuns | PlainMessage | undefined): boolean { + return proto3.util.equals(JobNextRuns, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobNextRunsRequest + */ +export class GetJobNextRunsRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobNextRunsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobNextRunsRequest { + return new GetJobNextRunsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobNextRunsRequest { + return new GetJobNextRunsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobNextRunsRequest { + return new GetJobNextRunsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobNextRunsRequest | PlainMessage | undefined, b: GetJobNextRunsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobNextRunsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobNextRunsResponse + */ +export class GetJobNextRunsResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.JobNextRuns next_runs = 1; + */ + nextRuns?: JobNextRuns; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobNextRunsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "next_runs", kind: "message", T: JobNextRuns }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobNextRunsResponse { + return new GetJobNextRunsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobNextRunsResponse { + return new GetJobNextRunsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobNextRunsResponse { + return new GetJobNextRunsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobNextRunsResponse | PlainMessage | undefined, b: GetJobNextRunsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobNextRunsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobStatusRequest + */ +export class GetJobStatusRequest extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobStatusRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusRequest { + return new GetJobStatusRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusRequest { + return new GetJobStatusRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobStatusRequest { + return new GetJobStatusRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobStatusRequest | PlainMessage | undefined, b: GetJobStatusRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobStatusRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobStatusResponse + */ +export class GetJobStatusResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.JobStatus status = 1; + */ + status = JobStatus.UNSPECIFIED; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobStatusResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "status", kind: "enum", T: proto3.getEnumType(JobStatus) }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusResponse { + return new GetJobStatusResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusResponse { + return new GetJobStatusResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobStatusResponse { + return new GetJobStatusResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobStatusResponse | PlainMessage | undefined, b: GetJobStatusResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobStatusResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobStatusRecord + */ +export class JobStatusRecord extends Message { + /** + * @generated from field: string job_id = 1; + */ + jobId = ""; + + /** + * @generated from field: mgmt.v1alpha1.JobStatus status = 2; + */ + status = JobStatus.UNSPECIFIED; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobStatusRecord"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "status", kind: "enum", T: proto3.getEnumType(JobStatus) }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobStatusRecord { + return new JobStatusRecord().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobStatusRecord { + return new JobStatusRecord().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobStatusRecord { + return new JobStatusRecord().fromJsonString(jsonString, options); + } + + static equals(a: JobStatusRecord | PlainMessage | undefined, b: JobStatusRecord | PlainMessage | undefined): boolean { + return proto3.util.equals(JobStatusRecord, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobStatusesRequest + */ +export class GetJobStatusesRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobStatusesRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusesRequest { + return new GetJobStatusesRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusesRequest { + return new GetJobStatusesRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobStatusesRequest { + return new GetJobStatusesRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobStatusesRequest | PlainMessage | undefined, b: GetJobStatusesRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobStatusesRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobStatusesResponse + */ +export class GetJobStatusesResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.JobStatusRecord statuses = 1; + */ + statuses: JobStatusRecord[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobStatusesResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "statuses", kind: "message", T: JobStatusRecord, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobStatusesResponse { + return new GetJobStatusesResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobStatusesResponse { + return new GetJobStatusesResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobStatusesResponse { + return new GetJobStatusesResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobStatusesResponse | PlainMessage | undefined, b: GetJobStatusesResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobStatusesResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ActivityFailure + */ +export class ActivityFailure extends Message { + /** + * @generated from field: string message = 1; + */ + message = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ActivityFailure"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "message", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ActivityFailure { + return new ActivityFailure().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ActivityFailure { + return new ActivityFailure().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ActivityFailure { + return new ActivityFailure().fromJsonString(jsonString, options); + } + + static equals(a: ActivityFailure | PlainMessage | undefined, b: ActivityFailure | PlainMessage | undefined): boolean { + return proto3.util.equals(ActivityFailure, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.PendingActivity + */ +export class PendingActivity extends Message { + /** + * @generated from field: mgmt.v1alpha1.ActivityStatus status = 1; + */ + status = ActivityStatus.UNSPECIFIED; + + /** + * @generated from field: string activity_name = 2; + */ + activityName = ""; + + /** + * @generated from field: optional mgmt.v1alpha1.ActivityFailure last_failure = 3; + */ + lastFailure?: ActivityFailure; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.PendingActivity"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "status", kind: "enum", T: proto3.getEnumType(ActivityStatus) }, + { no: 2, name: "activity_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "last_failure", kind: "message", T: ActivityFailure, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): PendingActivity { + return new PendingActivity().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): PendingActivity { + return new PendingActivity().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): PendingActivity { + return new PendingActivity().fromJsonString(jsonString, options); + } + + static equals(a: PendingActivity | PlainMessage | undefined, b: PendingActivity | PlainMessage | undefined): boolean { + return proto3.util.equals(PendingActivity, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRun + */ +export class JobRun extends Message { + /** + * The id of the job run. This will currently be equivalent to the temporal workflow id + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * The unique identifier of the job id this run is associated with + * + * @generated from field: string job_id = 2; + */ + jobId = ""; + + /** + * The name of the job run. + * + * @generated from field: string name = 3; + */ + name = ""; + + /** + * the status of the job run + * + * @generated from field: mgmt.v1alpha1.JobRunStatus status = 4; + */ + status = JobRunStatus.UNSPECIFIED; + + /** + * A timestamp of when the run started + * + * @generated from field: google.protobuf.Timestamp started_at = 6; + */ + startedAt?: Timestamp; + + /** + * Available if the run completed or has not yet been archived by the system + * + * @generated from field: optional google.protobuf.Timestamp completed_at = 7; + */ + completedAt?: Timestamp; + + /** + * Pending activities are only returned when retrieving a specific job run and will not be returned when requesting job runs in list format + * + * @generated from field: repeated mgmt.v1alpha1.PendingActivity pending_activities = 8; + */ + pendingActivities: PendingActivity[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRun"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "status", kind: "enum", T: proto3.getEnumType(JobRunStatus) }, + { no: 6, name: "started_at", kind: "message", T: Timestamp }, + { no: 7, name: "completed_at", kind: "message", T: Timestamp, opt: true }, + { no: 8, name: "pending_activities", kind: "message", T: PendingActivity, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRun { + return new JobRun().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRun { + return new JobRun().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRun { + return new JobRun().fromJsonString(jsonString, options); + } + + static equals(a: JobRun | PlainMessage | undefined, b: JobRun | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRun, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRunEventTaskError + */ +export class JobRunEventTaskError extends Message { + /** + * @generated from field: string message = 1; + */ + message = ""; + + /** + * @generated from field: string retry_state = 2; + */ + retryState = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRunEventTaskError"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "message", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "retry_state", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventTaskError { + return new JobRunEventTaskError().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventTaskError { + return new JobRunEventTaskError().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRunEventTaskError { + return new JobRunEventTaskError().fromJsonString(jsonString, options); + } + + static equals(a: JobRunEventTaskError | PlainMessage | undefined, b: JobRunEventTaskError | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRunEventTaskError, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRunEventTask + */ +export class JobRunEventTask extends Message { + /** + * @generated from field: int64 id = 1; + */ + id = protoInt64.zero; + + /** + * @generated from field: string type = 2; + */ + type = ""; + + /** + * @generated from field: google.protobuf.Timestamp event_time = 3; + */ + eventTime?: Timestamp; + + /** + * @generated from field: mgmt.v1alpha1.JobRunEventTaskError error = 4; + */ + error?: JobRunEventTaskError; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRunEventTask"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 2, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "event_time", kind: "message", T: Timestamp }, + { no: 4, name: "error", kind: "message", T: JobRunEventTaskError }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventTask { + return new JobRunEventTask().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventTask { + return new JobRunEventTask().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRunEventTask { + return new JobRunEventTask().fromJsonString(jsonString, options); + } + + static equals(a: JobRunEventTask | PlainMessage | undefined, b: JobRunEventTask | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRunEventTask, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRunSyncMetadata + */ +export class JobRunSyncMetadata extends Message { + /** + * @generated from field: string schema = 1; + */ + schema = ""; + + /** + * @generated from field: string table = 2; + */ + table = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRunSyncMetadata"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "schema", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "table", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRunSyncMetadata { + return new JobRunSyncMetadata().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRunSyncMetadata { + return new JobRunSyncMetadata().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRunSyncMetadata { + return new JobRunSyncMetadata().fromJsonString(jsonString, options); + } + + static equals(a: JobRunSyncMetadata | PlainMessage | undefined, b: JobRunSyncMetadata | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRunSyncMetadata, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRunEventMetadata + */ +export class JobRunEventMetadata extends Message { + /** + * @generated from oneof mgmt.v1alpha1.JobRunEventMetadata.metadata + */ + metadata: { + /** + * @generated from field: mgmt.v1alpha1.JobRunSyncMetadata sync_metadata = 1; + */ + value: JobRunSyncMetadata; + case: "syncMetadata"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRunEventMetadata"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "sync_metadata", kind: "message", T: JobRunSyncMetadata, oneof: "metadata" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEventMetadata { + return new JobRunEventMetadata().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEventMetadata { + return new JobRunEventMetadata().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRunEventMetadata { + return new JobRunEventMetadata().fromJsonString(jsonString, options); + } + + static equals(a: JobRunEventMetadata | PlainMessage | undefined, b: JobRunEventMetadata | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRunEventMetadata, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.JobRunEvent + */ +export class JobRunEvent extends Message { + /** + * @generated from field: int64 id = 1; + */ + id = protoInt64.zero; + + /** + * @generated from field: string type = 2; + */ + type = ""; + + /** + * @generated from field: google.protobuf.Timestamp start_time = 3; + */ + startTime?: Timestamp; + + /** + * @generated from field: google.protobuf.Timestamp close_time = 4; + */ + closeTime?: Timestamp; + + /** + * @generated from field: mgmt.v1alpha1.JobRunEventMetadata metadata = 5; + */ + metadata?: JobRunEventMetadata; + + /** + * @generated from field: repeated mgmt.v1alpha1.JobRunEventTask tasks = 6; + */ + tasks: JobRunEventTask[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.JobRunEvent"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 2, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "start_time", kind: "message", T: Timestamp }, + { no: 4, name: "close_time", kind: "message", T: Timestamp }, + { no: 5, name: "metadata", kind: "message", T: JobRunEventMetadata }, + { no: 6, name: "tasks", kind: "message", T: JobRunEventTask, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): JobRunEvent { + return new JobRunEvent().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): JobRunEvent { + return new JobRunEvent().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): JobRunEvent { + return new JobRunEvent().fromJsonString(jsonString, options); + } + + static equals(a: JobRunEvent | PlainMessage | undefined, b: JobRunEvent | PlainMessage | undefined): boolean { + return proto3.util.equals(JobRunEvent, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunEventsRequest + */ +export class GetJobRunEventsRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunEventsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunEventsRequest { + return new GetJobRunEventsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunEventsRequest { + return new GetJobRunEventsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunEventsRequest { + return new GetJobRunEventsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunEventsRequest | PlainMessage | undefined, b: GetJobRunEventsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunEventsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunEventsResponse + */ +export class GetJobRunEventsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.JobRunEvent events = 1; + */ + events: JobRunEvent[] = []; + + /** + * @generated from field: bool is_run_complete = 2; + */ + isRunComplete = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunEventsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "events", kind: "message", T: JobRunEvent, repeated: true }, + { no: 2, name: "is_run_complete", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunEventsResponse { + return new GetJobRunEventsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunEventsResponse { + return new GetJobRunEventsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunEventsResponse { + return new GetJobRunEventsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunEventsResponse | PlainMessage | undefined, b: GetJobRunEventsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunEventsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobRunRequest + */ +export class DeleteJobRunRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobRunRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRunRequest { + return new DeleteJobRunRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRunRequest { + return new DeleteJobRunRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobRunRequest { + return new DeleteJobRunRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobRunRequest | PlainMessage | undefined, b: DeleteJobRunRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobRunRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteJobRunResponse + */ +export class DeleteJobRunResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteJobRunResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteJobRunResponse { + return new DeleteJobRunResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteJobRunResponse { + return new DeleteJobRunResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteJobRunResponse { + return new DeleteJobRunResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteJobRunResponse | PlainMessage | undefined, b: DeleteJobRunResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteJobRunResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TerminateJobRunRequest + */ +export class TerminateJobRunRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TerminateJobRunRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TerminateJobRunRequest { + return new TerminateJobRunRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TerminateJobRunRequest { + return new TerminateJobRunRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TerminateJobRunRequest { + return new TerminateJobRunRequest().fromJsonString(jsonString, options); + } + + static equals(a: TerminateJobRunRequest | PlainMessage | undefined, b: TerminateJobRunRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(TerminateJobRunRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TerminateJobRunResponse + */ +export class TerminateJobRunResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TerminateJobRunResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TerminateJobRunResponse { + return new TerminateJobRunResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TerminateJobRunResponse { + return new TerminateJobRunResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TerminateJobRunResponse { + return new TerminateJobRunResponse().fromJsonString(jsonString, options); + } + + static equals(a: TerminateJobRunResponse | PlainMessage | undefined, b: TerminateJobRunResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(TerminateJobRunResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunLogsStreamRequest + */ +export class GetJobRunLogsStreamRequest extends Message { + /** + * @generated from field: string job_run_id = 1; + */ + jobRunId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + /** + * The time window in which to retrieve the logs + * + * @generated from field: mgmt.v1alpha1.LogWindow window = 3; + */ + window = LogWindow.NO_TIME_UNSPECIFIED; + + /** + * Whether or not to tail the stream. Note: only works with k8s-pods and is not currently supported with Loki logs + * + * @generated from field: bool should_tail = 4; + */ + shouldTail = false; + + /** + * Optionally provide a max log limit + * + * @generated from field: optional int64 max_log_lines = 5; + */ + maxLogLines?: bigint; + + /** + * Provide a list of log levels to filter by. If any of these are UNSPECIFIED, all log levels are returned. + * + * @generated from field: repeated mgmt.v1alpha1.LogLevel log_levels = 6; + */ + logLevels: LogLevel[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunLogsStreamRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job_run_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "window", kind: "enum", T: proto3.getEnumType(LogWindow) }, + { no: 4, name: "should_tail", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 5, name: "max_log_lines", kind: "scalar", T: 3 /* ScalarType.INT64 */, opt: true }, + { no: 6, name: "log_levels", kind: "enum", T: proto3.getEnumType(LogLevel), repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunLogsStreamRequest { + return new GetJobRunLogsStreamRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunLogsStreamRequest { + return new GetJobRunLogsStreamRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunLogsStreamRequest { + return new GetJobRunLogsStreamRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunLogsStreamRequest | PlainMessage | undefined, b: GetJobRunLogsStreamRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunLogsStreamRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetJobRunLogsStreamResponse + */ +export class GetJobRunLogsStreamResponse extends Message { + /** + * @generated from field: string log_line = 1; + */ + logLine = ""; + + /** + * @generated from field: optional google.protobuf.Timestamp timestamp = 2; + */ + timestamp?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetJobRunLogsStreamResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "log_line", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "timestamp", kind: "message", T: Timestamp, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetJobRunLogsStreamResponse { + return new GetJobRunLogsStreamResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetJobRunLogsStreamResponse { + return new GetJobRunLogsStreamResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetJobRunLogsStreamResponse { + return new GetJobRunLogsStreamResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetJobRunLogsStreamResponse | PlainMessage | undefined, b: GetJobRunLogsStreamResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetJobRunLogsStreamResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobWorkflowOptionsRequest + */ +export class SetJobWorkflowOptionsRequest extends Message { + /** + * The unique identifier of the job + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * The workflow options object. The entire object must be provided and will fully overwrite the previous result + * + * @generated from field: mgmt.v1alpha1.WorkflowOptions worfklow_options = 2; + */ + worfklowOptions?: WorkflowOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobWorkflowOptionsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "worfklow_options", kind: "message", T: WorkflowOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobWorkflowOptionsRequest { + return new SetJobWorkflowOptionsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobWorkflowOptionsRequest { + return new SetJobWorkflowOptionsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobWorkflowOptionsRequest { + return new SetJobWorkflowOptionsRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetJobWorkflowOptionsRequest | PlainMessage | undefined, b: SetJobWorkflowOptionsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobWorkflowOptionsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobWorkflowOptionsResponse + */ +export class SetJobWorkflowOptionsResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobWorkflowOptionsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobWorkflowOptionsResponse { + return new SetJobWorkflowOptionsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobWorkflowOptionsResponse { + return new SetJobWorkflowOptionsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobWorkflowOptionsResponse { + return new SetJobWorkflowOptionsResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetJobWorkflowOptionsResponse | PlainMessage | undefined, b: SetJobWorkflowOptionsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobWorkflowOptionsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobSyncOptionsRequest + */ +export class SetJobSyncOptionsRequest extends Message { + /** + * The unique identifier of the job + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * The sync options object. The entire object must be provided and will fully overwrite the previous result + * + * @generated from field: mgmt.v1alpha1.ActivityOptions sync_options = 2; + */ + syncOptions?: ActivityOptions; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobSyncOptionsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "sync_options", kind: "message", T: ActivityOptions }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSyncOptionsRequest { + return new SetJobSyncOptionsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSyncOptionsRequest { + return new SetJobSyncOptionsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobSyncOptionsRequest { + return new SetJobSyncOptionsRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetJobSyncOptionsRequest | PlainMessage | undefined, b: SetJobSyncOptionsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobSyncOptionsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetJobSyncOptionsResponse + */ +export class SetJobSyncOptionsResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.Job job = 1; + */ + job?: Job; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetJobSyncOptionsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "job", kind: "message", T: Job }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetJobSyncOptionsResponse { + return new SetJobSyncOptionsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetJobSyncOptionsResponse { + return new SetJobSyncOptionsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetJobSyncOptionsResponse { + return new SetJobSyncOptionsResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetJobSyncOptionsResponse | PlainMessage | undefined, b: SetJobSyncOptionsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetJobSyncOptionsResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts new file mode 100644 index 0000000000..34bc6a22e6 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_connect.ts @@ -0,0 +1,39 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/metrics.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { GetDailyMetricCountRequest, GetDailyMetricCountResponse, GetMetricCountRequest, GetMetricCountResponse } from "./metrics_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * @generated from service mgmt.v1alpha1.MetricsService + */ +export const MetricsService = { + typeName: "mgmt.v1alpha1.MetricsService", + methods: { + /** + * Retrieve a timed range of records + * + * @generated from rpc mgmt.v1alpha1.MetricsService.GetDailyMetricCount + */ + getDailyMetricCount: { + name: "GetDailyMetricCount", + I: GetDailyMetricCountRequest, + O: GetDailyMetricCountResponse, + kind: MethodKind.Unary, + }, + /** + * For the given metric and time range, returns the total count found + * + * @generated from rpc mgmt.v1alpha1.MetricsService.GetMetricCount + */ + getMetricCount: { + name: "GetMetricCount", + I: GetMetricCountRequest, + O: GetMetricCountResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts new file mode 100644 index 0000000000..bb153bf706 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/metrics_pb.ts @@ -0,0 +1,398 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/metrics.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; + +/** + * @generated from enum mgmt.v1alpha1.RangedMetricName + */ +export enum RangedMetricName { + /** + * If unspecified, an error will be thrown + * + * @generated from enum value: RANGED_METRIC_NAME_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * The input_received metric + * + * @generated from enum value: RANGED_METRIC_NAME_INPUT_RECEIVED = 1; + */ + INPUT_RECEIVED = 1, +} +// Retrieve enum metadata with: proto3.getEnumType(RangedMetricName) +proto3.util.setEnumType(RangedMetricName, "mgmt.v1alpha1.RangedMetricName", [ + { no: 0, name: "RANGED_METRIC_NAME_UNSPECIFIED" }, + { no: 1, name: "RANGED_METRIC_NAME_INPUT_RECEIVED" }, +]); + +/** + * Represents a whole or partial calendar date, such as a birthday. The time of + * day and time zone are either specified elsewhere or are insignificant. The + * date is relative to the Gregorian Calendar. This can represent one of the + * following: + * + * * A full date, with non-zero year, month, and day values + * * A month and day value, with a zero year, such as an anniversary + * * A year on its own, with zero month and day values + * * A year and month value, with a zero day, such as a credit card expiration + * date + * + * Related types are [google.type.TimeOfDay][google.type.TimeOfDay] and + * `google.protobuf.Timestamp`. + * + * @generated from message mgmt.v1alpha1.Date + */ +export class Date extends Message { + /** + * Year of the date. Must be from 1 to 9999, or 0 to specify a date without + * a year. + * + * @generated from field: uint32 year = 1; + */ + year = 0; + + /** + * Month of a year. Must be from 1 to 12, or 0 to specify a year without a + * month and day. + * + * @generated from field: uint32 month = 2; + */ + month = 0; + + /** + * Day of a month. Must be from 1 to 31 and valid for the year and month, or 0 + * to specify a year by itself or a year and month where the day isn't + * significant. + * + * @generated from field: uint32 day = 3; + */ + day = 0; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.Date"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "year", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, + { no: 2, name: "month", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, + { no: 3, name: "day", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): Date { + return new Date().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): Date { + return new Date().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): Date { + return new Date().fromJsonString(jsonString, options); + } + + static equals(a: Date | PlainMessage | undefined, b: Date | PlainMessage | undefined): boolean { + return proto3.util.equals(Date, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetDailyMetricCountRequest + */ +export class GetDailyMetricCountRequest extends Message { + /** + * The start day + * + * @generated from field: mgmt.v1alpha1.Date start = 1; + */ + start?: Date; + + /** + * The end day + * + * @generated from field: mgmt.v1alpha1.Date end = 2; + */ + end?: Date; + + /** + * The metric to return + * + * @generated from field: mgmt.v1alpha1.RangedMetricName metric = 3; + */ + metric = RangedMetricName.UNSPECIFIED; + + /** + * @generated from oneof mgmt.v1alpha1.GetDailyMetricCountRequest.identifier + */ + identifier: { + /** + * The account identifier that will be used to filter by + * + * @generated from field: string account_id = 4; + */ + value: string; + case: "accountId"; + } | { + /** + * The job identifier that will be used to filter by + * + * @generated from field: string job_id = 5; + */ + value: string; + case: "jobId"; + } | { + /** + * The run identifier that will be used to filter by + * + * @generated from field: string run_id = 6; + */ + value: string; + case: "runId"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetDailyMetricCountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "start", kind: "message", T: Date }, + { no: 2, name: "end", kind: "message", T: Date }, + { no: 3, name: "metric", kind: "enum", T: proto3.getEnumType(RangedMetricName) }, + { no: 4, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + { no: 5, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + { no: 6, name: "run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetDailyMetricCountRequest { + return new GetDailyMetricCountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetDailyMetricCountRequest { + return new GetDailyMetricCountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetDailyMetricCountRequest { + return new GetDailyMetricCountRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetDailyMetricCountRequest | PlainMessage | undefined, b: GetDailyMetricCountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetDailyMetricCountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetDailyMetricCountResponse + */ +export class GetDailyMetricCountResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.DayResult results = 1; + */ + results: DayResult[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetDailyMetricCountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "results", kind: "message", T: DayResult, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetDailyMetricCountResponse { + return new GetDailyMetricCountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetDailyMetricCountResponse { + return new GetDailyMetricCountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetDailyMetricCountResponse { + return new GetDailyMetricCountResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetDailyMetricCountResponse | PlainMessage | undefined, b: GetDailyMetricCountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetDailyMetricCountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DayResult + */ +export class DayResult extends Message { + /** + * @generated from field: mgmt.v1alpha1.Date date = 1; + */ + date?: Date; + + /** + * @generated from field: uint64 count = 2; + */ + count = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DayResult"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "date", kind: "message", T: Date }, + { no: 2, name: "count", kind: "scalar", T: 4 /* ScalarType.UINT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DayResult { + return new DayResult().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DayResult { + return new DayResult().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DayResult { + return new DayResult().fromJsonString(jsonString, options); + } + + static equals(a: DayResult | PlainMessage | undefined, b: DayResult | PlainMessage | undefined): boolean { + return proto3.util.equals(DayResult, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetMetricCountRequest + */ +export class GetMetricCountRequest extends Message { + /** + * The start time + * + * @generated from field: google.protobuf.Timestamp start = 1; + */ + start?: Timestamp; + + /** + * The end time + * + * @generated from field: google.protobuf.Timestamp end = 2; + */ + end?: Timestamp; + + /** + * The metric to return + * + * @generated from field: mgmt.v1alpha1.RangedMetricName metric = 3; + */ + metric = RangedMetricName.UNSPECIFIED; + + /** + * @generated from oneof mgmt.v1alpha1.GetMetricCountRequest.identifier + */ + identifier: { + /** + * The account identifier that will be used to filter by + * + * @generated from field: string account_id = 4; + */ + value: string; + case: "accountId"; + } | { + /** + * The job identifier that will be used to filter by + * + * @generated from field: string job_id = 5; + */ + value: string; + case: "jobId"; + } | { + /** + * The run identifier that will be used to filter by + * + * @generated from field: string run_id = 6; + */ + value: string; + case: "runId"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetMetricCountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "start", kind: "message", T: Timestamp }, + { no: 2, name: "end", kind: "message", T: Timestamp }, + { no: 3, name: "metric", kind: "enum", T: proto3.getEnumType(RangedMetricName) }, + { no: 4, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + { no: 5, name: "job_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + { no: 6, name: "run_id", kind: "scalar", T: 9 /* ScalarType.STRING */, oneof: "identifier" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetMetricCountRequest { + return new GetMetricCountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetMetricCountRequest { + return new GetMetricCountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetMetricCountRequest { + return new GetMetricCountRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetMetricCountRequest | PlainMessage | undefined, b: GetMetricCountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetMetricCountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetMetricCountResponse + */ +export class GetMetricCountResponse extends Message { + /** + * The summed up count of the metric based on the input query and timerange specified + * + * @generated from field: uint64 count = 1; + */ + count = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetMetricCountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "count", kind: "scalar", T: 4 /* ScalarType.UINT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetMetricCountResponse { + return new GetMetricCountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetMetricCountResponse { + return new GetMetricCountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetMetricCountResponse { + return new GetMetricCountResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetMetricCountResponse | PlainMessage | undefined, b: GetMetricCountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetMetricCountResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts new file mode 100644 index 0000000000..9ecd9ab022 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_connect.ts @@ -0,0 +1,107 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/transformer.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { CreateUserDefinedTransformerRequest, CreateUserDefinedTransformerResponse, DeleteUserDefinedTransformerRequest, DeleteUserDefinedTransformerResponse, GetSystemTransformerBySourceRequest, GetSystemTransformerBySourceResponse, GetSystemTransformersRequest, GetSystemTransformersResponse, GetUserDefinedTransformerByIdRequest, GetUserDefinedTransformerByIdResponse, GetUserDefinedTransformersRequest, GetUserDefinedTransformersResponse, IsTransformerNameAvailableRequest, IsTransformerNameAvailableResponse, UpdateUserDefinedTransformerRequest, UpdateUserDefinedTransformerResponse, ValidateUserJavascriptCodeRequest, ValidateUserJavascriptCodeResponse, ValidateUserRegexCodeRequest, ValidateUserRegexCodeResponse } from "./transformer_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * @generated from service mgmt.v1alpha1.TransformersService + */ +export const TransformersService = { + typeName: "mgmt.v1alpha1.TransformersService", + methods: { + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.GetSystemTransformers + */ + getSystemTransformers: { + name: "GetSystemTransformers", + I: GetSystemTransformersRequest, + O: GetSystemTransformersResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.GetSystemTransformerBySource + */ + getSystemTransformerBySource: { + name: "GetSystemTransformerBySource", + I: GetSystemTransformerBySourceRequest, + O: GetSystemTransformerBySourceResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.GetUserDefinedTransformers + */ + getUserDefinedTransformers: { + name: "GetUserDefinedTransformers", + I: GetUserDefinedTransformersRequest, + O: GetUserDefinedTransformersResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.GetUserDefinedTransformerById + */ + getUserDefinedTransformerById: { + name: "GetUserDefinedTransformerById", + I: GetUserDefinedTransformerByIdRequest, + O: GetUserDefinedTransformerByIdResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.CreateUserDefinedTransformer + */ + createUserDefinedTransformer: { + name: "CreateUserDefinedTransformer", + I: CreateUserDefinedTransformerRequest, + O: CreateUserDefinedTransformerResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.DeleteUserDefinedTransformer + */ + deleteUserDefinedTransformer: { + name: "DeleteUserDefinedTransformer", + I: DeleteUserDefinedTransformerRequest, + O: DeleteUserDefinedTransformerResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.UpdateUserDefinedTransformer + */ + updateUserDefinedTransformer: { + name: "UpdateUserDefinedTransformer", + I: UpdateUserDefinedTransformerRequest, + O: UpdateUserDefinedTransformerResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.IsTransformerNameAvailable + */ + isTransformerNameAvailable: { + name: "IsTransformerNameAvailable", + I: IsTransformerNameAvailableRequest, + O: IsTransformerNameAvailableResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.ValidateUserJavascriptCode + */ + validateUserJavascriptCode: { + name: "ValidateUserJavascriptCode", + I: ValidateUserJavascriptCodeRequest, + O: ValidateUserJavascriptCodeResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.TransformersService.ValidateUserRegexCode + */ + validateUserRegexCode: { + name: "ValidateUserRegexCode", + I: ValidateUserRegexCodeRequest, + O: ValidateUserRegexCodeResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts new file mode 100644 index 0000000000..206238d0b0 --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/transformer_pb.ts @@ -0,0 +1,3180 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/transformer.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, protoInt64, Timestamp } from "@bufbuild/protobuf"; + +/** + * @generated from enum mgmt.v1alpha1.TransformerSource + */ +export enum TransformerSource { + /** + * @generated from enum value: TRANSFORMER_SOURCE_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_PASSTHROUGH = 1; + */ + PASSTHROUGH = 1, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_DEFAULT = 2; + */ + GENERATE_DEFAULT = 2, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT = 3; + */ + TRANSFORM_JAVASCRIPT = 3, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_EMAIL = 4; + */ + GENERATE_EMAIL = 4, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_EMAIL = 5; + */ + TRANSFORM_EMAIL = 5, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_BOOL = 6; + */ + GENERATE_BOOL = 6, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER = 7; + */ + GENERATE_CARD_NUMBER = 7, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CITY = 8; + */ + GENERATE_CITY = 8, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER = 9; + */ + GENERATE_E164_PHONE_NUMBER = 9, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FIRST_NAME = 10; + */ + GENERATE_FIRST_NAME = 10, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FLOAT64 = 11; + */ + GENERATE_FLOAT64 = 11, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS = 12; + */ + GENERATE_FULL_ADDRESS = 12, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_FULL_NAME = 13; + */ + GENERATE_FULL_NAME = 13, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_GENDER = 14; + */ + GENERATE_GENDER = 14, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER = 15; + */ + GENERATE_INT64_PHONE_NUMBER = 15, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_INT64 = 16; + */ + GENERATE_INT64 = 16, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64 = 17; + */ + GENERATE_RANDOM_INT64 = 17, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_LAST_NAME = 18; + */ + GENERATE_LAST_NAME = 18, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_SHA256HASH = 19; + */ + GENERATE_SHA256HASH = 19, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_SSN = 20; + */ + GENERATE_SSN = 20, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STATE = 21; + */ + GENERATE_STATE = 21, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS = 22; + */ + GENERATE_STREET_ADDRESS = 22, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER = 23; + */ + GENERATE_STRING_PHONE_NUMBER = 23, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_STRING = 24; + */ + GENERATE_STRING = 24, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING = 25; + */ + GENERATE_RANDOM_STRING = 25, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP = 26; + */ + GENERATE_UNIXTIMESTAMP = 26, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_USERNAME = 27; + */ + GENERATE_USERNAME = 27, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP = 28; + */ + GENERATE_UTCTIMESTAMP = 28, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_UUID = 29; + */ + GENERATE_UUID = 29, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_ZIPCODE = 30; + */ + GENERATE_ZIPCODE = 30, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER = 31; + */ + TRANSFORM_E164_PHONE_NUMBER = 31, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME = 32; + */ + TRANSFORM_FIRST_NAME = 32, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FLOAT64 = 33; + */ + TRANSFORM_FLOAT64 = 33, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME = 34; + */ + TRANSFORM_FULL_NAME = 34, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER = 35; + */ + TRANSFORM_INT64_PHONE_NUMBER = 35, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_INT64 = 36; + */ + TRANSFORM_INT64 = 36, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME = 37; + */ + TRANSFORM_LAST_NAME = 37, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER = 38; + */ + TRANSFORM_PHONE_NUMBER = 38, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_STRING = 39; + */ + TRANSFORM_STRING = 39, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_NULL = 40; + */ + GENERATE_NULL = 40, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_CATEGORICAL = 42; + */ + GENERATE_CATEGORICAL = 42, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE = 43; + */ + TRANSFORM_CHARACTER_SCRAMBLE = 43, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_USER_DEFINED = 44; + */ + USER_DEFINED = 44, + + /** + * @generated from enum value: TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT = 45; + */ + GENERATE_JAVASCRIPT = 45, +} +// Retrieve enum metadata with: proto3.getEnumType(TransformerSource) +proto3.util.setEnumType(TransformerSource, "mgmt.v1alpha1.TransformerSource", [ + { no: 0, name: "TRANSFORMER_SOURCE_UNSPECIFIED" }, + { no: 1, name: "TRANSFORMER_SOURCE_PASSTHROUGH" }, + { no: 2, name: "TRANSFORMER_SOURCE_GENERATE_DEFAULT" }, + { no: 3, name: "TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT" }, + { no: 4, name: "TRANSFORMER_SOURCE_GENERATE_EMAIL" }, + { no: 5, name: "TRANSFORMER_SOURCE_TRANSFORM_EMAIL" }, + { no: 6, name: "TRANSFORMER_SOURCE_GENERATE_BOOL" }, + { no: 7, name: "TRANSFORMER_SOURCE_GENERATE_CARD_NUMBER" }, + { no: 8, name: "TRANSFORMER_SOURCE_GENERATE_CITY" }, + { no: 9, name: "TRANSFORMER_SOURCE_GENERATE_E164_PHONE_NUMBER" }, + { no: 10, name: "TRANSFORMER_SOURCE_GENERATE_FIRST_NAME" }, + { no: 11, name: "TRANSFORMER_SOURCE_GENERATE_FLOAT64" }, + { no: 12, name: "TRANSFORMER_SOURCE_GENERATE_FULL_ADDRESS" }, + { no: 13, name: "TRANSFORMER_SOURCE_GENERATE_FULL_NAME" }, + { no: 14, name: "TRANSFORMER_SOURCE_GENERATE_GENDER" }, + { no: 15, name: "TRANSFORMER_SOURCE_GENERATE_INT64_PHONE_NUMBER" }, + { no: 16, name: "TRANSFORMER_SOURCE_GENERATE_INT64" }, + { no: 17, name: "TRANSFORMER_SOURCE_GENERATE_RANDOM_INT64" }, + { no: 18, name: "TRANSFORMER_SOURCE_GENERATE_LAST_NAME" }, + { no: 19, name: "TRANSFORMER_SOURCE_GENERATE_SHA256HASH" }, + { no: 20, name: "TRANSFORMER_SOURCE_GENERATE_SSN" }, + { no: 21, name: "TRANSFORMER_SOURCE_GENERATE_STATE" }, + { no: 22, name: "TRANSFORMER_SOURCE_GENERATE_STREET_ADDRESS" }, + { no: 23, name: "TRANSFORMER_SOURCE_GENERATE_STRING_PHONE_NUMBER" }, + { no: 24, name: "TRANSFORMER_SOURCE_GENERATE_STRING" }, + { no: 25, name: "TRANSFORMER_SOURCE_GENERATE_RANDOM_STRING" }, + { no: 26, name: "TRANSFORMER_SOURCE_GENERATE_UNIXTIMESTAMP" }, + { no: 27, name: "TRANSFORMER_SOURCE_GENERATE_USERNAME" }, + { no: 28, name: "TRANSFORMER_SOURCE_GENERATE_UTCTIMESTAMP" }, + { no: 29, name: "TRANSFORMER_SOURCE_GENERATE_UUID" }, + { no: 30, name: "TRANSFORMER_SOURCE_GENERATE_ZIPCODE" }, + { no: 31, name: "TRANSFORMER_SOURCE_TRANSFORM_E164_PHONE_NUMBER" }, + { no: 32, name: "TRANSFORMER_SOURCE_TRANSFORM_FIRST_NAME" }, + { no: 33, name: "TRANSFORMER_SOURCE_TRANSFORM_FLOAT64" }, + { no: 34, name: "TRANSFORMER_SOURCE_TRANSFORM_FULL_NAME" }, + { no: 35, name: "TRANSFORMER_SOURCE_TRANSFORM_INT64_PHONE_NUMBER" }, + { no: 36, name: "TRANSFORMER_SOURCE_TRANSFORM_INT64" }, + { no: 37, name: "TRANSFORMER_SOURCE_TRANSFORM_LAST_NAME" }, + { no: 38, name: "TRANSFORMER_SOURCE_TRANSFORM_PHONE_NUMBER" }, + { no: 39, name: "TRANSFORMER_SOURCE_TRANSFORM_STRING" }, + { no: 40, name: "TRANSFORMER_SOURCE_GENERATE_NULL" }, + { no: 42, name: "TRANSFORMER_SOURCE_GENERATE_CATEGORICAL" }, + { no: 43, name: "TRANSFORMER_SOURCE_TRANSFORM_CHARACTER_SCRAMBLE" }, + { no: 44, name: "TRANSFORMER_SOURCE_USER_DEFINED" }, + { no: 45, name: "TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT" }, +]); + +/** + * @generated from enum mgmt.v1alpha1.TransformerDataType + */ +export enum TransformerDataType { + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_STRING = 1; + */ + STRING = 1, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_INT64 = 2; + */ + INT64 = 2, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_BOOLEAN = 3; + */ + BOOLEAN = 3, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_FLOAT64 = 4; + */ + FLOAT64 = 4, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_NULL = 5; + */ + NULL = 5, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_ANY = 6; + */ + ANY = 6, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_TIME = 7; + */ + TIME = 7, + + /** + * @generated from enum value: TRANSFORMER_DATA_TYPE_UUID = 8; + */ + UUID = 8, +} +// Retrieve enum metadata with: proto3.getEnumType(TransformerDataType) +proto3.util.setEnumType(TransformerDataType, "mgmt.v1alpha1.TransformerDataType", [ + { no: 0, name: "TRANSFORMER_DATA_TYPE_UNSPECIFIED" }, + { no: 1, name: "TRANSFORMER_DATA_TYPE_STRING" }, + { no: 2, name: "TRANSFORMER_DATA_TYPE_INT64" }, + { no: 3, name: "TRANSFORMER_DATA_TYPE_BOOLEAN" }, + { no: 4, name: "TRANSFORMER_DATA_TYPE_FLOAT64" }, + { no: 5, name: "TRANSFORMER_DATA_TYPE_NULL" }, + { no: 6, name: "TRANSFORMER_DATA_TYPE_ANY" }, + { no: 7, name: "TRANSFORMER_DATA_TYPE_TIME" }, + { no: 8, name: "TRANSFORMER_DATA_TYPE_UUID" }, +]); + +/** + * @generated from enum mgmt.v1alpha1.SupportedJobType + */ +export enum SupportedJobType { + /** + * @generated from enum value: SUPPORTED_JOB_TYPE_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: SUPPORTED_JOB_TYPE_SYNC = 1; + */ + SYNC = 1, + + /** + * @generated from enum value: SUPPORTED_JOB_TYPE_GENERATE = 2; + */ + GENERATE = 2, +} +// Retrieve enum metadata with: proto3.getEnumType(SupportedJobType) +proto3.util.setEnumType(SupportedJobType, "mgmt.v1alpha1.SupportedJobType", [ + { no: 0, name: "SUPPORTED_JOB_TYPE_UNSPECIFIED" }, + { no: 1, name: "SUPPORTED_JOB_TYPE_SYNC" }, + { no: 2, name: "SUPPORTED_JOB_TYPE_GENERATE" }, +]); + +/** + * @generated from message mgmt.v1alpha1.GetSystemTransformersRequest + */ +export class GetSystemTransformersRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemTransformersRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformersRequest { + return new GetSystemTransformersRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformersRequest { + return new GetSystemTransformersRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformersRequest { + return new GetSystemTransformersRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemTransformersRequest | PlainMessage | undefined, b: GetSystemTransformersRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemTransformersRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetSystemTransformersResponse + */ +export class GetSystemTransformersResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.SystemTransformer transformers = 1; + */ + transformers: SystemTransformer[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemTransformersResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformers", kind: "message", T: SystemTransformer, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformersResponse { + return new GetSystemTransformersResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformersResponse { + return new GetSystemTransformersResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformersResponse { + return new GetSystemTransformersResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemTransformersResponse | PlainMessage | undefined, b: GetSystemTransformersResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemTransformersResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetSystemTransformerBySourceRequest + */ +export class GetSystemTransformerBySourceRequest extends Message { + /** + * @generated from field: mgmt.v1alpha1.TransformerSource source = 1; + */ + source = TransformerSource.UNSPECIFIED; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemTransformerBySourceRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformerBySourceRequest { + return new GetSystemTransformerBySourceRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformerBySourceRequest { + return new GetSystemTransformerBySourceRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformerBySourceRequest { + return new GetSystemTransformerBySourceRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemTransformerBySourceRequest | PlainMessage | undefined, b: GetSystemTransformerBySourceRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemTransformerBySourceRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetSystemTransformerBySourceResponse + */ +export class GetSystemTransformerBySourceResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.SystemTransformer transformer = 1; + */ + transformer?: SystemTransformer; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemTransformerBySourceResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer", kind: "message", T: SystemTransformer }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemTransformerBySourceResponse { + return new GetSystemTransformerBySourceResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemTransformerBySourceResponse { + return new GetSystemTransformerBySourceResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemTransformerBySourceResponse { + return new GetSystemTransformerBySourceResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemTransformerBySourceResponse | PlainMessage | undefined, b: GetSystemTransformerBySourceResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemTransformerBySourceResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserDefinedTransformersRequest + */ +export class GetUserDefinedTransformersRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformersRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformersRequest { + return new GetUserDefinedTransformersRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformersRequest { + return new GetUserDefinedTransformersRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformersRequest { + return new GetUserDefinedTransformersRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetUserDefinedTransformersRequest | PlainMessage | undefined, b: GetUserDefinedTransformersRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserDefinedTransformersRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserDefinedTransformersResponse + */ +export class GetUserDefinedTransformersResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.UserDefinedTransformer transformers = 1; + */ + transformers: UserDefinedTransformer[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformersResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformers", kind: "message", T: UserDefinedTransformer, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformersResponse { + return new GetUserDefinedTransformersResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformersResponse { + return new GetUserDefinedTransformersResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformersResponse { + return new GetUserDefinedTransformersResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetUserDefinedTransformersResponse | PlainMessage | undefined, b: GetUserDefinedTransformersResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserDefinedTransformersResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest + */ +export class GetUserDefinedTransformerByIdRequest extends Message { + /** + * @generated from field: string transformer_id = 1; + */ + transformerId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformerByIdRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformerByIdRequest { + return new GetUserDefinedTransformerByIdRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformerByIdRequest { + return new GetUserDefinedTransformerByIdRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformerByIdRequest { + return new GetUserDefinedTransformerByIdRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetUserDefinedTransformerByIdRequest | PlainMessage | undefined, b: GetUserDefinedTransformerByIdRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserDefinedTransformerByIdRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse + */ +export class GetUserDefinedTransformerByIdResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; + */ + transformer?: UserDefinedTransformer; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserDefinedTransformerByIdResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserDefinedTransformerByIdResponse { + return new GetUserDefinedTransformerByIdResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserDefinedTransformerByIdResponse { + return new GetUserDefinedTransformerByIdResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserDefinedTransformerByIdResponse { + return new GetUserDefinedTransformerByIdResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetUserDefinedTransformerByIdResponse | PlainMessage | undefined, b: GetUserDefinedTransformerByIdResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserDefinedTransformerByIdResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateUserDefinedTransformerRequest + */ +export class CreateUserDefinedTransformerRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: string description = 3; + */ + description = ""; + + /** + * @deprecated + * + * @generated from field: string type = 4; + */ + type = ""; + + /** + * @generated from field: mgmt.v1alpha1.TransformerSource source = 5; + */ + source = TransformerSource.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerConfig transformer_config = 6; + */ + transformerConfig?: TransformerConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateUserDefinedTransformerRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, + { no: 6, name: "transformer_config", kind: "message", T: TransformerConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateUserDefinedTransformerRequest { + return new CreateUserDefinedTransformerRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateUserDefinedTransformerRequest { + return new CreateUserDefinedTransformerRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateUserDefinedTransformerRequest { + return new CreateUserDefinedTransformerRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateUserDefinedTransformerRequest | PlainMessage | undefined, b: CreateUserDefinedTransformerRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateUserDefinedTransformerRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateUserDefinedTransformerResponse + */ +export class CreateUserDefinedTransformerResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; + */ + transformer?: UserDefinedTransformer; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateUserDefinedTransformerResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateUserDefinedTransformerResponse { + return new CreateUserDefinedTransformerResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateUserDefinedTransformerResponse { + return new CreateUserDefinedTransformerResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateUserDefinedTransformerResponse { + return new CreateUserDefinedTransformerResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateUserDefinedTransformerResponse | PlainMessage | undefined, b: CreateUserDefinedTransformerResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateUserDefinedTransformerResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteUserDefinedTransformerRequest + */ +export class DeleteUserDefinedTransformerRequest extends Message { + /** + * @generated from field: string transformer_id = 1; + */ + transformerId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteUserDefinedTransformerRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteUserDefinedTransformerRequest { + return new DeleteUserDefinedTransformerRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteUserDefinedTransformerRequest { + return new DeleteUserDefinedTransformerRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteUserDefinedTransformerRequest { + return new DeleteUserDefinedTransformerRequest().fromJsonString(jsonString, options); + } + + static equals(a: DeleteUserDefinedTransformerRequest | PlainMessage | undefined, b: DeleteUserDefinedTransformerRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteUserDefinedTransformerRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.DeleteUserDefinedTransformerResponse + */ +export class DeleteUserDefinedTransformerResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.DeleteUserDefinedTransformerResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): DeleteUserDefinedTransformerResponse { + return new DeleteUserDefinedTransformerResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): DeleteUserDefinedTransformerResponse { + return new DeleteUserDefinedTransformerResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): DeleteUserDefinedTransformerResponse { + return new DeleteUserDefinedTransformerResponse().fromJsonString(jsonString, options); + } + + static equals(a: DeleteUserDefinedTransformerResponse | PlainMessage | undefined, b: DeleteUserDefinedTransformerResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(DeleteUserDefinedTransformerResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateUserDefinedTransformerRequest + */ +export class UpdateUserDefinedTransformerRequest extends Message { + /** + * @generated from field: string transformer_id = 1; + */ + transformerId = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: string description = 3; + */ + description = ""; + + /** + * @generated from field: mgmt.v1alpha1.TransformerConfig transformer_config = 4; + */ + transformerConfig?: TransformerConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateUserDefinedTransformerRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "transformer_config", kind: "message", T: TransformerConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateUserDefinedTransformerRequest { + return new UpdateUserDefinedTransformerRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateUserDefinedTransformerRequest { + return new UpdateUserDefinedTransformerRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateUserDefinedTransformerRequest { + return new UpdateUserDefinedTransformerRequest().fromJsonString(jsonString, options); + } + + static equals(a: UpdateUserDefinedTransformerRequest | PlainMessage | undefined, b: UpdateUserDefinedTransformerRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateUserDefinedTransformerRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UpdateUserDefinedTransformerResponse + */ +export class UpdateUserDefinedTransformerResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.UserDefinedTransformer transformer = 1; + */ + transformer?: UserDefinedTransformer; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UpdateUserDefinedTransformerResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "transformer", kind: "message", T: UserDefinedTransformer }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UpdateUserDefinedTransformerResponse { + return new UpdateUserDefinedTransformerResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UpdateUserDefinedTransformerResponse { + return new UpdateUserDefinedTransformerResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UpdateUserDefinedTransformerResponse { + return new UpdateUserDefinedTransformerResponse().fromJsonString(jsonString, options); + } + + static equals(a: UpdateUserDefinedTransformerResponse | PlainMessage | undefined, b: UpdateUserDefinedTransformerResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(UpdateUserDefinedTransformerResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsTransformerNameAvailableRequest + */ +export class IsTransformerNameAvailableRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string transformer_name = 2; + */ + transformerName = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsTransformerNameAvailableRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "transformer_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsTransformerNameAvailableRequest { + return new IsTransformerNameAvailableRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsTransformerNameAvailableRequest { + return new IsTransformerNameAvailableRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsTransformerNameAvailableRequest { + return new IsTransformerNameAvailableRequest().fromJsonString(jsonString, options); + } + + static equals(a: IsTransformerNameAvailableRequest | PlainMessage | undefined, b: IsTransformerNameAvailableRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(IsTransformerNameAvailableRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsTransformerNameAvailableResponse + */ +export class IsTransformerNameAvailableResponse extends Message { + /** + * @generated from field: bool is_available = 1; + */ + isAvailable = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsTransformerNameAvailableResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "is_available", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsTransformerNameAvailableResponse { + return new IsTransformerNameAvailableResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsTransformerNameAvailableResponse { + return new IsTransformerNameAvailableResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsTransformerNameAvailableResponse { + return new IsTransformerNameAvailableResponse().fromJsonString(jsonString, options); + } + + static equals(a: IsTransformerNameAvailableResponse | PlainMessage | undefined, b: IsTransformerNameAvailableResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(IsTransformerNameAvailableResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UserDefinedTransformer + */ +export class UserDefinedTransformer extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: string description = 3; + */ + description = ""; + + /** + * This property is readonly and is calculated based off the origin system transformer + * + * @generated from field: mgmt.v1alpha1.TransformerDataType data_type = 5; + */ + dataType = TransformerDataType.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerSource source = 6; + */ + source = TransformerSource.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerConfig config = 7; + */ + config?: TransformerConfig; + + /** + * @generated from field: google.protobuf.Timestamp created_at = 8; + */ + createdAt?: Timestamp; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 9; + */ + updatedAt?: Timestamp; + + /** + * @generated from field: string account_id = 10; + */ + accountId = ""; + + /** + * @generated from field: repeated mgmt.v1alpha1.TransformerDataType data_types = 11; + */ + dataTypes: TransformerDataType[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UserDefinedTransformer"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "data_type", kind: "enum", T: proto3.getEnumType(TransformerDataType) }, + { no: 6, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, + { no: 7, name: "config", kind: "message", T: TransformerConfig }, + { no: 8, name: "created_at", kind: "message", T: Timestamp }, + { no: 9, name: "updated_at", kind: "message", T: Timestamp }, + { no: 10, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 11, name: "data_types", kind: "enum", T: proto3.getEnumType(TransformerDataType), repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UserDefinedTransformer { + return new UserDefinedTransformer().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UserDefinedTransformer { + return new UserDefinedTransformer().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UserDefinedTransformer { + return new UserDefinedTransformer().fromJsonString(jsonString, options); + } + + static equals(a: UserDefinedTransformer | PlainMessage | undefined, b: UserDefinedTransformer | PlainMessage | undefined): boolean { + return proto3.util.equals(UserDefinedTransformer, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SystemTransformer + */ +export class SystemTransformer extends Message { + /** + * @generated from field: string name = 1; + */ + name = ""; + + /** + * @generated from field: string description = 2; + */ + description = ""; + + /** + * @generated from field: mgmt.v1alpha1.TransformerDataType data_type = 3; + */ + dataType = TransformerDataType.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerSource source = 4; + */ + source = TransformerSource.UNSPECIFIED; + + /** + * @generated from field: mgmt.v1alpha1.TransformerConfig config = 5; + */ + config?: TransformerConfig; + + /** + * @generated from field: repeated mgmt.v1alpha1.TransformerDataType data_types = 6; + */ + dataTypes: TransformerDataType[] = []; + + /** + * @generated from field: repeated mgmt.v1alpha1.SupportedJobType supported_job_types = 7; + */ + supportedJobTypes: SupportedJobType[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SystemTransformer"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "description", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "data_type", kind: "enum", T: proto3.getEnumType(TransformerDataType) }, + { no: 4, name: "source", kind: "enum", T: proto3.getEnumType(TransformerSource) }, + { no: 5, name: "config", kind: "message", T: TransformerConfig }, + { no: 6, name: "data_types", kind: "enum", T: proto3.getEnumType(TransformerDataType), repeated: true }, + { no: 7, name: "supported_job_types", kind: "enum", T: proto3.getEnumType(SupportedJobType), repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SystemTransformer { + return new SystemTransformer().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SystemTransformer { + return new SystemTransformer().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SystemTransformer { + return new SystemTransformer().fromJsonString(jsonString, options); + } + + static equals(a: SystemTransformer | PlainMessage | undefined, b: SystemTransformer | PlainMessage | undefined): boolean { + return proto3.util.equals(SystemTransformer, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformerConfig + */ +export class TransformerConfig extends Message { + /** + * @generated from oneof mgmt.v1alpha1.TransformerConfig.config + */ + config: { + /** + * @generated from field: mgmt.v1alpha1.GenerateEmail generate_email_config = 1; + */ + value: GenerateEmail; + case: "generateEmailConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformEmail transform_email_config = 2; + */ + value: TransformEmail; + case: "transformEmailConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateBool generate_bool_config = 3; + */ + value: GenerateBool; + case: "generateBoolConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateCardNumber generate_card_number_config = 4; + */ + value: GenerateCardNumber; + case: "generateCardNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateCity generate_city_config = 5; + */ + value: GenerateCity; + case: "generateCityConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateE164PhoneNumber generate_e164_phone_number_config = 6; + */ + value: GenerateE164PhoneNumber; + case: "generateE164PhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateFirstName generate_first_name_config = 7; + */ + value: GenerateFirstName; + case: "generateFirstNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateFloat64 generate_float64_config = 8; + */ + value: GenerateFloat64; + case: "generateFloat64Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateFullAddress generate_full_address_config = 9; + */ + value: GenerateFullAddress; + case: "generateFullAddressConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateFullName generate_full_name_config = 10; + */ + value: GenerateFullName; + case: "generateFullNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateGender generate_gender_config = 11; + */ + value: GenerateGender; + case: "generateGenderConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateInt64PhoneNumber generate_int64_phone_number_config = 12; + */ + value: GenerateInt64PhoneNumber; + case: "generateInt64PhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateInt64 generate_int64_config = 13; + */ + value: GenerateInt64; + case: "generateInt64Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateLastName generate_last_name_config = 14; + */ + value: GenerateLastName; + case: "generateLastNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateSha256Hash generate_sha256hash_config = 15; + */ + value: GenerateSha256Hash; + case: "generateSha256hashConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateSSN generate_ssn_config = 16; + */ + value: GenerateSSN; + case: "generateSsnConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateState generate_state_config = 17; + */ + value: GenerateState; + case: "generateStateConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateStreetAddress generate_street_address_config = 18; + */ + value: GenerateStreetAddress; + case: "generateStreetAddressConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateStringPhoneNumber generate_string_phone_number_config = 19; + */ + value: GenerateStringPhoneNumber; + case: "generateStringPhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateString generate_string_config = 20; + */ + value: GenerateString; + case: "generateStringConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateUnixTimestamp generate_unixtimestamp_config = 21; + */ + value: GenerateUnixTimestamp; + case: "generateUnixtimestampConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateUsername generate_username_config = 22; + */ + value: GenerateUsername; + case: "generateUsernameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateUtcTimestamp generate_utctimestamp_config = 23; + */ + value: GenerateUtcTimestamp; + case: "generateUtctimestampConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateUuid generate_uuid_config = 24; + */ + value: GenerateUuid; + case: "generateUuidConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateZipcode generate_zipcode_config = 25; + */ + value: GenerateZipcode; + case: "generateZipcodeConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformE164PhoneNumber transform_e164_phone_number_config = 26; + */ + value: TransformE164PhoneNumber; + case: "transformE164PhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformFirstName transform_first_name_config = 27; + */ + value: TransformFirstName; + case: "transformFirstNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformFloat64 transform_float64_config = 28; + */ + value: TransformFloat64; + case: "transformFloat64Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformFullName transform_full_name_config = 29; + */ + value: TransformFullName; + case: "transformFullNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformInt64PhoneNumber transform_int64_phone_number_config = 30; + */ + value: TransformInt64PhoneNumber; + case: "transformInt64PhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformInt64 transform_int64_config = 31; + */ + value: TransformInt64; + case: "transformInt64Config"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformLastName transform_last_name_config = 32; + */ + value: TransformLastName; + case: "transformLastNameConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformPhoneNumber transform_phone_number_config = 33; + */ + value: TransformPhoneNumber; + case: "transformPhoneNumberConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformString transform_string_config = 34; + */ + value: TransformString; + case: "transformStringConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.Passthrough passthrough_config = 35; + */ + value: Passthrough; + case: "passthroughConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.Null nullconfig = 36; + */ + value: Null; + case: "nullconfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.UserDefinedTransformerConfig user_defined_transformer_config = 37; + */ + value: UserDefinedTransformerConfig; + case: "userDefinedTransformerConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateDefault generate_default_config = 38; + */ + value: GenerateDefault; + case: "generateDefaultConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformJavascript transform_javascript_config = 39; + */ + value: TransformJavascript; + case: "transformJavascriptConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateCategorical generate_categorical_config = 40; + */ + value: GenerateCategorical; + case: "generateCategoricalConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.TransformCharacterScramble transform_character_scramble_config = 41; + */ + value: TransformCharacterScramble; + case: "transformCharacterScrambleConfig"; + } | { + /** + * @generated from field: mgmt.v1alpha1.GenerateJavascript generate_javascript_config = 42; + */ + value: GenerateJavascript; + case: "generateJavascriptConfig"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformerConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "generate_email_config", kind: "message", T: GenerateEmail, oneof: "config" }, + { no: 2, name: "transform_email_config", kind: "message", T: TransformEmail, oneof: "config" }, + { no: 3, name: "generate_bool_config", kind: "message", T: GenerateBool, oneof: "config" }, + { no: 4, name: "generate_card_number_config", kind: "message", T: GenerateCardNumber, oneof: "config" }, + { no: 5, name: "generate_city_config", kind: "message", T: GenerateCity, oneof: "config" }, + { no: 6, name: "generate_e164_phone_number_config", kind: "message", T: GenerateE164PhoneNumber, oneof: "config" }, + { no: 7, name: "generate_first_name_config", kind: "message", T: GenerateFirstName, oneof: "config" }, + { no: 8, name: "generate_float64_config", kind: "message", T: GenerateFloat64, oneof: "config" }, + { no: 9, name: "generate_full_address_config", kind: "message", T: GenerateFullAddress, oneof: "config" }, + { no: 10, name: "generate_full_name_config", kind: "message", T: GenerateFullName, oneof: "config" }, + { no: 11, name: "generate_gender_config", kind: "message", T: GenerateGender, oneof: "config" }, + { no: 12, name: "generate_int64_phone_number_config", kind: "message", T: GenerateInt64PhoneNumber, oneof: "config" }, + { no: 13, name: "generate_int64_config", kind: "message", T: GenerateInt64, oneof: "config" }, + { no: 14, name: "generate_last_name_config", kind: "message", T: GenerateLastName, oneof: "config" }, + { no: 15, name: "generate_sha256hash_config", kind: "message", T: GenerateSha256Hash, oneof: "config" }, + { no: 16, name: "generate_ssn_config", kind: "message", T: GenerateSSN, oneof: "config" }, + { no: 17, name: "generate_state_config", kind: "message", T: GenerateState, oneof: "config" }, + { no: 18, name: "generate_street_address_config", kind: "message", T: GenerateStreetAddress, oneof: "config" }, + { no: 19, name: "generate_string_phone_number_config", kind: "message", T: GenerateStringPhoneNumber, oneof: "config" }, + { no: 20, name: "generate_string_config", kind: "message", T: GenerateString, oneof: "config" }, + { no: 21, name: "generate_unixtimestamp_config", kind: "message", T: GenerateUnixTimestamp, oneof: "config" }, + { no: 22, name: "generate_username_config", kind: "message", T: GenerateUsername, oneof: "config" }, + { no: 23, name: "generate_utctimestamp_config", kind: "message", T: GenerateUtcTimestamp, oneof: "config" }, + { no: 24, name: "generate_uuid_config", kind: "message", T: GenerateUuid, oneof: "config" }, + { no: 25, name: "generate_zipcode_config", kind: "message", T: GenerateZipcode, oneof: "config" }, + { no: 26, name: "transform_e164_phone_number_config", kind: "message", T: TransformE164PhoneNumber, oneof: "config" }, + { no: 27, name: "transform_first_name_config", kind: "message", T: TransformFirstName, oneof: "config" }, + { no: 28, name: "transform_float64_config", kind: "message", T: TransformFloat64, oneof: "config" }, + { no: 29, name: "transform_full_name_config", kind: "message", T: TransformFullName, oneof: "config" }, + { no: 30, name: "transform_int64_phone_number_config", kind: "message", T: TransformInt64PhoneNumber, oneof: "config" }, + { no: 31, name: "transform_int64_config", kind: "message", T: TransformInt64, oneof: "config" }, + { no: 32, name: "transform_last_name_config", kind: "message", T: TransformLastName, oneof: "config" }, + { no: 33, name: "transform_phone_number_config", kind: "message", T: TransformPhoneNumber, oneof: "config" }, + { no: 34, name: "transform_string_config", kind: "message", T: TransformString, oneof: "config" }, + { no: 35, name: "passthrough_config", kind: "message", T: Passthrough, oneof: "config" }, + { no: 36, name: "nullconfig", kind: "message", T: Null, oneof: "config" }, + { no: 37, name: "user_defined_transformer_config", kind: "message", T: UserDefinedTransformerConfig, oneof: "config" }, + { no: 38, name: "generate_default_config", kind: "message", T: GenerateDefault, oneof: "config" }, + { no: 39, name: "transform_javascript_config", kind: "message", T: TransformJavascript, oneof: "config" }, + { no: 40, name: "generate_categorical_config", kind: "message", T: GenerateCategorical, oneof: "config" }, + { no: 41, name: "transform_character_scramble_config", kind: "message", T: TransformCharacterScramble, oneof: "config" }, + { no: 42, name: "generate_javascript_config", kind: "message", T: GenerateJavascript, oneof: "config" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformerConfig { + return new TransformerConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformerConfig { + return new TransformerConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformerConfig { + return new TransformerConfig().fromJsonString(jsonString, options); + } + + static equals(a: TransformerConfig | PlainMessage | undefined, b: TransformerConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformerConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateEmail + */ +export class GenerateEmail extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateEmail"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateEmail { + return new GenerateEmail().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateEmail { + return new GenerateEmail().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateEmail { + return new GenerateEmail().fromJsonString(jsonString, options); + } + + static equals(a: GenerateEmail | PlainMessage | undefined, b: GenerateEmail | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateEmail, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformEmail + */ +export class TransformEmail extends Message { + /** + * @generated from field: bool preserve_domain = 1; + */ + preserveDomain = false; + + /** + * @generated from field: bool preserve_length = 2; + */ + preserveLength = false; + + /** + * @generated from field: repeated string excluded_domains = 3; + */ + excludedDomains: string[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformEmail"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_domain", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "excluded_domains", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformEmail { + return new TransformEmail().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformEmail { + return new TransformEmail().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformEmail { + return new TransformEmail().fromJsonString(jsonString, options); + } + + static equals(a: TransformEmail | PlainMessage | undefined, b: TransformEmail | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformEmail, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateBool + */ +export class GenerateBool extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateBool"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateBool { + return new GenerateBool().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateBool { + return new GenerateBool().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateBool { + return new GenerateBool().fromJsonString(jsonString, options); + } + + static equals(a: GenerateBool | PlainMessage | undefined, b: GenerateBool | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateBool, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateCardNumber + */ +export class GenerateCardNumber extends Message { + /** + * @generated from field: bool valid_luhn = 1; + */ + validLuhn = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateCardNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "valid_luhn", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCardNumber { + return new GenerateCardNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCardNumber { + return new GenerateCardNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateCardNumber { + return new GenerateCardNumber().fromJsonString(jsonString, options); + } + + static equals(a: GenerateCardNumber | PlainMessage | undefined, b: GenerateCardNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateCardNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateCity + */ +export class GenerateCity extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateCity"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCity { + return new GenerateCity().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCity { + return new GenerateCity().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateCity { + return new GenerateCity().fromJsonString(jsonString, options); + } + + static equals(a: GenerateCity | PlainMessage | undefined, b: GenerateCity | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateCity, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateDefault + */ +export class GenerateDefault extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateDefault"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateDefault { + return new GenerateDefault().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateDefault { + return new GenerateDefault().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateDefault { + return new GenerateDefault().fromJsonString(jsonString, options); + } + + static equals(a: GenerateDefault | PlainMessage | undefined, b: GenerateDefault | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateDefault, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateE164PhoneNumber + */ +export class GenerateE164PhoneNumber extends Message { + /** + * @generated from field: int64 min = 1; + */ + min = protoInt64.zero; + + /** + * @generated from field: int64 max = 2; + */ + max = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateE164PhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 2, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateE164PhoneNumber { + return new GenerateE164PhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateE164PhoneNumber { + return new GenerateE164PhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateE164PhoneNumber { + return new GenerateE164PhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: GenerateE164PhoneNumber | PlainMessage | undefined, b: GenerateE164PhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateE164PhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateFirstName + */ +export class GenerateFirstName extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateFirstName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFirstName { + return new GenerateFirstName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFirstName { + return new GenerateFirstName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateFirstName { + return new GenerateFirstName().fromJsonString(jsonString, options); + } + + static equals(a: GenerateFirstName | PlainMessage | undefined, b: GenerateFirstName | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateFirstName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateFloat64 + */ +export class GenerateFloat64 extends Message { + /** + * @generated from field: bool randomize_sign = 1; + */ + randomizeSign = false; + + /** + * @generated from field: double min = 2; + */ + min = 0; + + /** + * @generated from field: double max = 3; + */ + max = 0; + + /** + * @generated from field: int64 precision = 4; + */ + precision = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateFloat64"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "randomize_sign", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "min", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, + { no: 3, name: "max", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, + { no: 4, name: "precision", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFloat64 { + return new GenerateFloat64().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFloat64 { + return new GenerateFloat64().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateFloat64 { + return new GenerateFloat64().fromJsonString(jsonString, options); + } + + static equals(a: GenerateFloat64 | PlainMessage | undefined, b: GenerateFloat64 | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateFloat64, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateFullAddress + */ +export class GenerateFullAddress extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateFullAddress"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFullAddress { + return new GenerateFullAddress().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFullAddress { + return new GenerateFullAddress().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateFullAddress { + return new GenerateFullAddress().fromJsonString(jsonString, options); + } + + static equals(a: GenerateFullAddress | PlainMessage | undefined, b: GenerateFullAddress | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateFullAddress, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateFullName + */ +export class GenerateFullName extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateFullName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateFullName { + return new GenerateFullName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateFullName { + return new GenerateFullName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateFullName { + return new GenerateFullName().fromJsonString(jsonString, options); + } + + static equals(a: GenerateFullName | PlainMessage | undefined, b: GenerateFullName | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateFullName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateGender + */ +export class GenerateGender extends Message { + /** + * @generated from field: bool abbreviate = 1; + */ + abbreviate = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateGender"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "abbreviate", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateGender { + return new GenerateGender().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateGender { + return new GenerateGender().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateGender { + return new GenerateGender().fromJsonString(jsonString, options); + } + + static equals(a: GenerateGender | PlainMessage | undefined, b: GenerateGender | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateGender, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateInt64PhoneNumber + */ +export class GenerateInt64PhoneNumber extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateInt64PhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateInt64PhoneNumber { + return new GenerateInt64PhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateInt64PhoneNumber { + return new GenerateInt64PhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateInt64PhoneNumber { + return new GenerateInt64PhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: GenerateInt64PhoneNumber | PlainMessage | undefined, b: GenerateInt64PhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateInt64PhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateInt64 + */ +export class GenerateInt64 extends Message { + /** + * @generated from field: bool randomize_sign = 1; + */ + randomizeSign = false; + + /** + * @generated from field: int64 min = 2; + */ + min = protoInt64.zero; + + /** + * @generated from field: int64 max = 3; + */ + max = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateInt64"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "randomize_sign", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 3, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateInt64 { + return new GenerateInt64().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateInt64 { + return new GenerateInt64().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateInt64 { + return new GenerateInt64().fromJsonString(jsonString, options); + } + + static equals(a: GenerateInt64 | PlainMessage | undefined, b: GenerateInt64 | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateInt64, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateLastName + */ +export class GenerateLastName extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateLastName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateLastName { + return new GenerateLastName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateLastName { + return new GenerateLastName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateLastName { + return new GenerateLastName().fromJsonString(jsonString, options); + } + + static equals(a: GenerateLastName | PlainMessage | undefined, b: GenerateLastName | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateLastName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateSha256Hash + */ +export class GenerateSha256Hash extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateSha256Hash"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSha256Hash { + return new GenerateSha256Hash().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSha256Hash { + return new GenerateSha256Hash().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateSha256Hash { + return new GenerateSha256Hash().fromJsonString(jsonString, options); + } + + static equals(a: GenerateSha256Hash | PlainMessage | undefined, b: GenerateSha256Hash | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateSha256Hash, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateSSN + */ +export class GenerateSSN extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateSSN"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateSSN { + return new GenerateSSN().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateSSN { + return new GenerateSSN().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateSSN { + return new GenerateSSN().fromJsonString(jsonString, options); + } + + static equals(a: GenerateSSN | PlainMessage | undefined, b: GenerateSSN | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateSSN, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateState + */ +export class GenerateState extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateState"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateState { + return new GenerateState().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateState { + return new GenerateState().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateState { + return new GenerateState().fromJsonString(jsonString, options); + } + + static equals(a: GenerateState | PlainMessage | undefined, b: GenerateState | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateState, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateStreetAddress + */ +export class GenerateStreetAddress extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateStreetAddress"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateStreetAddress { + return new GenerateStreetAddress().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateStreetAddress { + return new GenerateStreetAddress().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateStreetAddress { + return new GenerateStreetAddress().fromJsonString(jsonString, options); + } + + static equals(a: GenerateStreetAddress | PlainMessage | undefined, b: GenerateStreetAddress | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateStreetAddress, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateStringPhoneNumber + */ +export class GenerateStringPhoneNumber extends Message { + /** + * @generated from field: int64 min = 2; + */ + min = protoInt64.zero; + + /** + * @generated from field: int64 max = 3; + */ + max = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateStringPhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 2, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 3, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateStringPhoneNumber { + return new GenerateStringPhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateStringPhoneNumber { + return new GenerateStringPhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateStringPhoneNumber { + return new GenerateStringPhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: GenerateStringPhoneNumber | PlainMessage | undefined, b: GenerateStringPhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateStringPhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateString + */ +export class GenerateString extends Message { + /** + * @generated from field: int64 min = 1; + */ + min = protoInt64.zero; + + /** + * @generated from field: int64 max = 2; + */ + max = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateString"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 2, name: "max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateString { + return new GenerateString().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateString { + return new GenerateString().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateString { + return new GenerateString().fromJsonString(jsonString, options); + } + + static equals(a: GenerateString | PlainMessage | undefined, b: GenerateString | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateString, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateUnixTimestamp + */ +export class GenerateUnixTimestamp extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateUnixTimestamp"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUnixTimestamp { + return new GenerateUnixTimestamp().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUnixTimestamp { + return new GenerateUnixTimestamp().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateUnixTimestamp { + return new GenerateUnixTimestamp().fromJsonString(jsonString, options); + } + + static equals(a: GenerateUnixTimestamp | PlainMessage | undefined, b: GenerateUnixTimestamp | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateUnixTimestamp, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateUsername + */ +export class GenerateUsername extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateUsername"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUsername { + return new GenerateUsername().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUsername { + return new GenerateUsername().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateUsername { + return new GenerateUsername().fromJsonString(jsonString, options); + } + + static equals(a: GenerateUsername | PlainMessage | undefined, b: GenerateUsername | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateUsername, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateUtcTimestamp + */ +export class GenerateUtcTimestamp extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateUtcTimestamp"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUtcTimestamp { + return new GenerateUtcTimestamp().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUtcTimestamp { + return new GenerateUtcTimestamp().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateUtcTimestamp { + return new GenerateUtcTimestamp().fromJsonString(jsonString, options); + } + + static equals(a: GenerateUtcTimestamp | PlainMessage | undefined, b: GenerateUtcTimestamp | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateUtcTimestamp, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateUuid + */ +export class GenerateUuid extends Message { + /** + * @generated from field: bool include_hyphens = 1; + */ + includeHyphens = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateUuid"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "include_hyphens", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateUuid { + return new GenerateUuid().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateUuid { + return new GenerateUuid().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateUuid { + return new GenerateUuid().fromJsonString(jsonString, options); + } + + static equals(a: GenerateUuid | PlainMessage | undefined, b: GenerateUuid | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateUuid, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateZipcode + */ +export class GenerateZipcode extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateZipcode"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateZipcode { + return new GenerateZipcode().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateZipcode { + return new GenerateZipcode().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateZipcode { + return new GenerateZipcode().fromJsonString(jsonString, options); + } + + static equals(a: GenerateZipcode | PlainMessage | undefined, b: GenerateZipcode | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateZipcode, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformE164PhoneNumber + */ +export class TransformE164PhoneNumber extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformE164PhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformE164PhoneNumber { + return new TransformE164PhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformE164PhoneNumber { + return new TransformE164PhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformE164PhoneNumber { + return new TransformE164PhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: TransformE164PhoneNumber | PlainMessage | undefined, b: TransformE164PhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformE164PhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformFirstName + */ +export class TransformFirstName extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformFirstName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformFirstName { + return new TransformFirstName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformFirstName { + return new TransformFirstName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformFirstName { + return new TransformFirstName().fromJsonString(jsonString, options); + } + + static equals(a: TransformFirstName | PlainMessage | undefined, b: TransformFirstName | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformFirstName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformFloat64 + */ +export class TransformFloat64 extends Message { + /** + * @generated from field: double randomization_range_min = 1; + */ + randomizationRangeMin = 0; + + /** + * @generated from field: double randomization_range_max = 2; + */ + randomizationRangeMax = 0; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformFloat64"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "randomization_range_min", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, + { no: 2, name: "randomization_range_max", kind: "scalar", T: 1 /* ScalarType.DOUBLE */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformFloat64 { + return new TransformFloat64().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformFloat64 { + return new TransformFloat64().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformFloat64 { + return new TransformFloat64().fromJsonString(jsonString, options); + } + + static equals(a: TransformFloat64 | PlainMessage | undefined, b: TransformFloat64 | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformFloat64, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformFullName + */ +export class TransformFullName extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformFullName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformFullName { + return new TransformFullName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformFullName { + return new TransformFullName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformFullName { + return new TransformFullName().fromJsonString(jsonString, options); + } + + static equals(a: TransformFullName | PlainMessage | undefined, b: TransformFullName | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformFullName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformInt64PhoneNumber + */ +export class TransformInt64PhoneNumber extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformInt64PhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformInt64PhoneNumber { + return new TransformInt64PhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformInt64PhoneNumber { + return new TransformInt64PhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformInt64PhoneNumber { + return new TransformInt64PhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: TransformInt64PhoneNumber | PlainMessage | undefined, b: TransformInt64PhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformInt64PhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformInt64 + */ +export class TransformInt64 extends Message { + /** + * @generated from field: int64 randomization_range_min = 1; + */ + randomizationRangeMin = protoInt64.zero; + + /** + * @generated from field: int64 randomization_range_max = 2; + */ + randomizationRangeMax = protoInt64.zero; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformInt64"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "randomization_range_min", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + { no: 2, name: "randomization_range_max", kind: "scalar", T: 3 /* ScalarType.INT64 */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformInt64 { + return new TransformInt64().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformInt64 { + return new TransformInt64().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformInt64 { + return new TransformInt64().fromJsonString(jsonString, options); + } + + static equals(a: TransformInt64 | PlainMessage | undefined, b: TransformInt64 | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformInt64, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformLastName + */ +export class TransformLastName extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformLastName"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformLastName { + return new TransformLastName().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformLastName { + return new TransformLastName().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformLastName { + return new TransformLastName().fromJsonString(jsonString, options); + } + + static equals(a: TransformLastName | PlainMessage | undefined, b: TransformLastName | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformLastName, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformPhoneNumber + */ +export class TransformPhoneNumber extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformPhoneNumber"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformPhoneNumber { + return new TransformPhoneNumber().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformPhoneNumber { + return new TransformPhoneNumber().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformPhoneNumber { + return new TransformPhoneNumber().fromJsonString(jsonString, options); + } + + static equals(a: TransformPhoneNumber | PlainMessage | undefined, b: TransformPhoneNumber | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformPhoneNumber, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformString + */ +export class TransformString extends Message { + /** + * @generated from field: bool preserve_length = 1; + */ + preserveLength = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformString"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "preserve_length", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformString { + return new TransformString().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformString { + return new TransformString().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformString { + return new TransformString().fromJsonString(jsonString, options); + } + + static equals(a: TransformString | PlainMessage | undefined, b: TransformString | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformString, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.Passthrough + */ +export class Passthrough extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.Passthrough"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): Passthrough { + return new Passthrough().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): Passthrough { + return new Passthrough().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): Passthrough { + return new Passthrough().fromJsonString(jsonString, options); + } + + static equals(a: Passthrough | PlainMessage | undefined, b: Passthrough | PlainMessage | undefined): boolean { + return proto3.util.equals(Passthrough, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.Null + */ +export class Null extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.Null"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): Null { + return new Null().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): Null { + return new Null().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): Null { + return new Null().fromJsonString(jsonString, options); + } + + static equals(a: Null | PlainMessage | undefined, b: Null | PlainMessage | undefined): boolean { + return proto3.util.equals(Null, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformJavascript + */ +export class TransformJavascript extends Message { + /** + * @generated from field: string code = 1; + */ + code = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformJavascript"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformJavascript { + return new TransformJavascript().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformJavascript { + return new TransformJavascript().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformJavascript { + return new TransformJavascript().fromJsonString(jsonString, options); + } + + static equals(a: TransformJavascript | PlainMessage | undefined, b: TransformJavascript | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformJavascript, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UserDefinedTransformerConfig + */ +export class UserDefinedTransformerConfig extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UserDefinedTransformerConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UserDefinedTransformerConfig { + return new UserDefinedTransformerConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UserDefinedTransformerConfig { + return new UserDefinedTransformerConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UserDefinedTransformerConfig { + return new UserDefinedTransformerConfig().fromJsonString(jsonString, options); + } + + static equals(a: UserDefinedTransformerConfig | PlainMessage | undefined, b: UserDefinedTransformerConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(UserDefinedTransformerConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ValidateUserJavascriptCodeRequest + */ +export class ValidateUserJavascriptCodeRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string code = 2; + */ + code = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ValidateUserJavascriptCodeRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserJavascriptCodeRequest { + return new ValidateUserJavascriptCodeRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserJavascriptCodeRequest { + return new ValidateUserJavascriptCodeRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ValidateUserJavascriptCodeRequest { + return new ValidateUserJavascriptCodeRequest().fromJsonString(jsonString, options); + } + + static equals(a: ValidateUserJavascriptCodeRequest | PlainMessage | undefined, b: ValidateUserJavascriptCodeRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(ValidateUserJavascriptCodeRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ValidateUserJavascriptCodeResponse + */ +export class ValidateUserJavascriptCodeResponse extends Message { + /** + * @generated from field: bool valid = 1; + */ + valid = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ValidateUserJavascriptCodeResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserJavascriptCodeResponse { + return new ValidateUserJavascriptCodeResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserJavascriptCodeResponse { + return new ValidateUserJavascriptCodeResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ValidateUserJavascriptCodeResponse { + return new ValidateUserJavascriptCodeResponse().fromJsonString(jsonString, options); + } + + static equals(a: ValidateUserJavascriptCodeResponse | PlainMessage | undefined, b: ValidateUserJavascriptCodeResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(ValidateUserJavascriptCodeResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateCategorical + */ +export class GenerateCategorical extends Message { + /** + * @generated from field: string categories = 1; + */ + categories = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateCategorical"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "categories", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateCategorical { + return new GenerateCategorical().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateCategorical { + return new GenerateCategorical().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateCategorical { + return new GenerateCategorical().fromJsonString(jsonString, options); + } + + static equals(a: GenerateCategorical | PlainMessage | undefined, b: GenerateCategorical | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateCategorical, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.TransformCharacterScramble + */ +export class TransformCharacterScramble extends Message { + /** + * a user provided regular expression that they wish to validate if it compiles in GO + * + * @generated from field: optional string user_provided_regex = 1; + */ + userProvidedRegex?: string; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.TransformCharacterScramble"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "user_provided_regex", kind: "scalar", T: 9 /* ScalarType.STRING */, opt: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): TransformCharacterScramble { + return new TransformCharacterScramble().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): TransformCharacterScramble { + return new TransformCharacterScramble().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): TransformCharacterScramble { + return new TransformCharacterScramble().fromJsonString(jsonString, options); + } + + static equals(a: TransformCharacterScramble | PlainMessage | undefined, b: TransformCharacterScramble | PlainMessage | undefined): boolean { + return proto3.util.equals(TransformCharacterScramble, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GenerateJavascript + */ +export class GenerateJavascript extends Message { + /** + * @generated from field: string code = 1; + */ + code = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GenerateJavascript"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "code", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GenerateJavascript { + return new GenerateJavascript().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GenerateJavascript { + return new GenerateJavascript().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GenerateJavascript { + return new GenerateJavascript().fromJsonString(jsonString, options); + } + + static equals(a: GenerateJavascript | PlainMessage | undefined, b: GenerateJavascript | PlainMessage | undefined): boolean { + return proto3.util.equals(GenerateJavascript, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ValidateUserRegexCodeRequest + */ +export class ValidateUserRegexCodeRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string user_provided_regex = 2; + */ + userProvidedRegex = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ValidateUserRegexCodeRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "user_provided_regex", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserRegexCodeRequest { + return new ValidateUserRegexCodeRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserRegexCodeRequest { + return new ValidateUserRegexCodeRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ValidateUserRegexCodeRequest { + return new ValidateUserRegexCodeRequest().fromJsonString(jsonString, options); + } + + static equals(a: ValidateUserRegexCodeRequest | PlainMessage | undefined, b: ValidateUserRegexCodeRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(ValidateUserRegexCodeRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ValidateUserRegexCodeResponse + */ +export class ValidateUserRegexCodeResponse extends Message { + /** + * @generated from field: bool valid = 1; + */ + valid = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ValidateUserRegexCodeResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "valid", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ValidateUserRegexCodeResponse { + return new ValidateUserRegexCodeResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ValidateUserRegexCodeResponse { + return new ValidateUserRegexCodeResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ValidateUserRegexCodeResponse { + return new ValidateUserRegexCodeResponse().fromJsonString(jsonString, options); + } + + static equals(a: ValidateUserRegexCodeResponse | PlainMessage | undefined, b: ValidateUserRegexCodeResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(ValidateUserRegexCodeResponse, a, b); + } +} + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts new file mode 100644 index 0000000000..9183e759ab --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_connect.ts @@ -0,0 +1,179 @@ +// @generated by protoc-gen-connect-es v1.4.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/user_account.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import { AcceptTeamAccountInviteRequest, AcceptTeamAccountInviteResponse, ConvertPersonalToTeamAccountRequest, ConvertPersonalToTeamAccountResponse, CreateTeamAccountRequest, CreateTeamAccountResponse, GetAccountOnboardingConfigRequest, GetAccountOnboardingConfigResponse, GetAccountTemporalConfigRequest, GetAccountTemporalConfigResponse, GetSystemInformationRequest, GetSystemInformationResponse, GetTeamAccountInvitesRequest, GetTeamAccountInvitesResponse, GetTeamAccountMembersRequest, GetTeamAccountMembersResponse, GetUserAccountsRequest, GetUserAccountsResponse, GetUserRequest, GetUserResponse, InviteUserToTeamAccountRequest, InviteUserToTeamAccountResponse, IsUserInAccountRequest, IsUserInAccountResponse, RemoveTeamAccountInviteRequest, RemoveTeamAccountInviteResponse, RemoveTeamAccountMemberRequest, RemoveTeamAccountMemberResponse, SetAccountOnboardingConfigRequest, SetAccountOnboardingConfigResponse, SetAccountTemporalConfigRequest, SetAccountTemporalConfigResponse, SetPersonalAccountRequest, SetPersonalAccountResponse, SetUserRequest, SetUserResponse } from "./user_account_pb.js"; +import { MethodKind } from "@bufbuild/protobuf"; + +/** + * @generated from service mgmt.v1alpha1.UserAccountService + */ +export const UserAccountService = { + typeName: "mgmt.v1alpha1.UserAccountService", + methods: { + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetUser + */ + getUser: { + name: "GetUser", + I: GetUserRequest, + O: GetUserResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.SetUser + */ + setUser: { + name: "SetUser", + I: SetUserRequest, + O: SetUserResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetUserAccounts + */ + getUserAccounts: { + name: "GetUserAccounts", + I: GetUserAccountsRequest, + O: GetUserAccountsResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.SetPersonalAccount + */ + setPersonalAccount: { + name: "SetPersonalAccount", + I: SetPersonalAccountRequest, + O: SetPersonalAccountResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.ConvertPersonalToTeamAccount + */ + convertPersonalToTeamAccount: { + name: "ConvertPersonalToTeamAccount", + I: ConvertPersonalToTeamAccountRequest, + O: ConvertPersonalToTeamAccountResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.CreateTeamAccount + */ + createTeamAccount: { + name: "CreateTeamAccount", + I: CreateTeamAccountRequest, + O: CreateTeamAccountResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.IsUserInAccount + */ + isUserInAccount: { + name: "IsUserInAccount", + I: IsUserInAccountRequest, + O: IsUserInAccountResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetAccountTemporalConfig + */ + getAccountTemporalConfig: { + name: "GetAccountTemporalConfig", + I: GetAccountTemporalConfigRequest, + O: GetAccountTemporalConfigResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.SetAccountTemporalConfig + */ + setAccountTemporalConfig: { + name: "SetAccountTemporalConfig", + I: SetAccountTemporalConfigRequest, + O: SetAccountTemporalConfigResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetTeamAccountMembers + */ + getTeamAccountMembers: { + name: "GetTeamAccountMembers", + I: GetTeamAccountMembersRequest, + O: GetTeamAccountMembersResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.RemoveTeamAccountMember + */ + removeTeamAccountMember: { + name: "RemoveTeamAccountMember", + I: RemoveTeamAccountMemberRequest, + O: RemoveTeamAccountMemberResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.InviteUserToTeamAccount + */ + inviteUserToTeamAccount: { + name: "InviteUserToTeamAccount", + I: InviteUserToTeamAccountRequest, + O: InviteUserToTeamAccountResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetTeamAccountInvites + */ + getTeamAccountInvites: { + name: "GetTeamAccountInvites", + I: GetTeamAccountInvitesRequest, + O: GetTeamAccountInvitesResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.RemoveTeamAccountInvite + */ + removeTeamAccountInvite: { + name: "RemoveTeamAccountInvite", + I: RemoveTeamAccountInviteRequest, + O: RemoveTeamAccountInviteResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.AcceptTeamAccountInvite + */ + acceptTeamAccountInvite: { + name: "AcceptTeamAccountInvite", + I: AcceptTeamAccountInviteRequest, + O: AcceptTeamAccountInviteResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetSystemInformation + */ + getSystemInformation: { + name: "GetSystemInformation", + I: GetSystemInformationRequest, + O: GetSystemInformationResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.GetAccountOnboardingConfig + */ + getAccountOnboardingConfig: { + name: "GetAccountOnboardingConfig", + I: GetAccountOnboardingConfigRequest, + O: GetAccountOnboardingConfigResponse, + kind: MethodKind.Unary, + }, + /** + * @generated from rpc mgmt.v1alpha1.UserAccountService.SetAccountOnboardingConfig + */ + setAccountOnboardingConfig: { + name: "SetAccountOnboardingConfig", + I: SetAccountOnboardingConfigRequest, + O: SetAccountOnboardingConfigResponse, + kind: MethodKind.Unary, + }, + } +} as const; + diff --git a/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts new file mode 100644 index 0000000000..2a384a897e --- /dev/null +++ b/frontend/packages/sdk/src/client/mgmt/v1alpha1/user_account_pb.ts @@ -0,0 +1,1659 @@ +// @generated by protoc-gen-es v1.8.0 with parameter "target=ts,import_extension=.js" +// @generated from file mgmt/v1alpha1/user_account.proto (package mgmt.v1alpha1, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, Timestamp } from "@bufbuild/protobuf"; + +/** + * @generated from enum mgmt.v1alpha1.UserAccountType + */ +export enum UserAccountType { + /** + * @generated from enum value: USER_ACCOUNT_TYPE_UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * @generated from enum value: USER_ACCOUNT_TYPE_PERSONAL = 1; + */ + PERSONAL = 1, + + /** + * @generated from enum value: USER_ACCOUNT_TYPE_TEAM = 2; + */ + TEAM = 2, + + /** + * @generated from enum value: USER_ACCOUNT_TYPE_ENTERPRISE = 3; + */ + ENTERPRISE = 3, +} +// Retrieve enum metadata with: proto3.getEnumType(UserAccountType) +proto3.util.setEnumType(UserAccountType, "mgmt.v1alpha1.UserAccountType", [ + { no: 0, name: "USER_ACCOUNT_TYPE_UNSPECIFIED" }, + { no: 1, name: "USER_ACCOUNT_TYPE_PERSONAL" }, + { no: 2, name: "USER_ACCOUNT_TYPE_TEAM" }, + { no: 3, name: "USER_ACCOUNT_TYPE_ENTERPRISE" }, +]); + +/** + * @generated from message mgmt.v1alpha1.GetUserRequest + */ +export class GetUserRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserRequest { + return new GetUserRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserRequest { + return new GetUserRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserRequest { + return new GetUserRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetUserRequest | PlainMessage | undefined, b: GetUserRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserResponse + */ +export class GetUserResponse extends Message { + /** + * @generated from field: string user_id = 1; + */ + userId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserResponse { + return new GetUserResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserResponse { + return new GetUserResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserResponse { + return new GetUserResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetUserResponse | PlainMessage | undefined, b: GetUserResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetUserRequest + */ +export class SetUserRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetUserRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetUserRequest { + return new SetUserRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetUserRequest { + return new SetUserRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetUserRequest { + return new SetUserRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetUserRequest | PlainMessage | undefined, b: SetUserRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetUserRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetUserResponse + */ +export class SetUserResponse extends Message { + /** + * @generated from field: string user_id = 1; + */ + userId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetUserResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetUserResponse { + return new SetUserResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetUserResponse { + return new SetUserResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetUserResponse { + return new SetUserResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetUserResponse | PlainMessage | undefined, b: SetUserResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetUserResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserAccountsRequest + */ +export class GetUserAccountsRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserAccountsRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserAccountsRequest { + return new GetUserAccountsRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserAccountsRequest { + return new GetUserAccountsRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserAccountsRequest { + return new GetUserAccountsRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetUserAccountsRequest | PlainMessage | undefined, b: GetUserAccountsRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserAccountsRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetUserAccountsResponse + */ +export class GetUserAccountsResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.UserAccount accounts = 1; + */ + accounts: UserAccount[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetUserAccountsResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "accounts", kind: "message", T: UserAccount, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetUserAccountsResponse { + return new GetUserAccountsResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetUserAccountsResponse { + return new GetUserAccountsResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetUserAccountsResponse { + return new GetUserAccountsResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetUserAccountsResponse | PlainMessage | undefined, b: GetUserAccountsResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetUserAccountsResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.UserAccount + */ +export class UserAccount extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: mgmt.v1alpha1.UserAccountType type = 3; + */ + type = UserAccountType.UNSPECIFIED; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.UserAccount"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "type", kind: "enum", T: proto3.getEnumType(UserAccountType) }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): UserAccount { + return new UserAccount().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): UserAccount { + return new UserAccount().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): UserAccount { + return new UserAccount().fromJsonString(jsonString, options); + } + + static equals(a: UserAccount | PlainMessage | undefined, b: UserAccount | PlainMessage | undefined): boolean { + return proto3.util.equals(UserAccount, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest + */ +export class ConvertPersonalToTeamAccountRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConvertPersonalToTeamAccountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConvertPersonalToTeamAccountRequest { + return new ConvertPersonalToTeamAccountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConvertPersonalToTeamAccountRequest { + return new ConvertPersonalToTeamAccountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConvertPersonalToTeamAccountRequest { + return new ConvertPersonalToTeamAccountRequest().fromJsonString(jsonString, options); + } + + static equals(a: ConvertPersonalToTeamAccountRequest | PlainMessage | undefined, b: ConvertPersonalToTeamAccountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(ConvertPersonalToTeamAccountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse + */ +export class ConvertPersonalToTeamAccountResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.ConvertPersonalToTeamAccountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ConvertPersonalToTeamAccountResponse { + return new ConvertPersonalToTeamAccountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ConvertPersonalToTeamAccountResponse { + return new ConvertPersonalToTeamAccountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ConvertPersonalToTeamAccountResponse { + return new ConvertPersonalToTeamAccountResponse().fromJsonString(jsonString, options); + } + + static equals(a: ConvertPersonalToTeamAccountResponse | PlainMessage | undefined, b: ConvertPersonalToTeamAccountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(ConvertPersonalToTeamAccountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetPersonalAccountRequest + */ +export class SetPersonalAccountRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetPersonalAccountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetPersonalAccountRequest { + return new SetPersonalAccountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetPersonalAccountRequest { + return new SetPersonalAccountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetPersonalAccountRequest { + return new SetPersonalAccountRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetPersonalAccountRequest | PlainMessage | undefined, b: SetPersonalAccountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetPersonalAccountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetPersonalAccountResponse + */ +export class SetPersonalAccountResponse extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetPersonalAccountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetPersonalAccountResponse { + return new SetPersonalAccountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetPersonalAccountResponse { + return new SetPersonalAccountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetPersonalAccountResponse { + return new SetPersonalAccountResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetPersonalAccountResponse | PlainMessage | undefined, b: SetPersonalAccountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetPersonalAccountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsUserInAccountRequest + */ +export class IsUserInAccountRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsUserInAccountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsUserInAccountRequest { + return new IsUserInAccountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsUserInAccountRequest { + return new IsUserInAccountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsUserInAccountRequest { + return new IsUserInAccountRequest().fromJsonString(jsonString, options); + } + + static equals(a: IsUserInAccountRequest | PlainMessage | undefined, b: IsUserInAccountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(IsUserInAccountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.IsUserInAccountResponse + */ +export class IsUserInAccountResponse extends Message { + /** + * @generated from field: bool ok = 1; + */ + ok = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.IsUserInAccountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "ok", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): IsUserInAccountResponse { + return new IsUserInAccountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): IsUserInAccountResponse { + return new IsUserInAccountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): IsUserInAccountResponse { + return new IsUserInAccountResponse().fromJsonString(jsonString, options); + } + + static equals(a: IsUserInAccountResponse | PlainMessage | undefined, b: IsUserInAccountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(IsUserInAccountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountTemporalConfigRequest + */ +export class GetAccountTemporalConfigRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountTemporalConfigRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountTemporalConfigRequest { + return new GetAccountTemporalConfigRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountTemporalConfigRequest { + return new GetAccountTemporalConfigRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountTemporalConfigRequest { + return new GetAccountTemporalConfigRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountTemporalConfigRequest | PlainMessage | undefined, b: GetAccountTemporalConfigRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountTemporalConfigRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountTemporalConfigResponse + */ +export class GetAccountTemporalConfigResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 1; + */ + config?: AccountTemporalConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountTemporalConfigResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "config", kind: "message", T: AccountTemporalConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountTemporalConfigResponse { + return new GetAccountTemporalConfigResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountTemporalConfigResponse { + return new GetAccountTemporalConfigResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountTemporalConfigResponse { + return new GetAccountTemporalConfigResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountTemporalConfigResponse | PlainMessage | undefined, b: GetAccountTemporalConfigResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountTemporalConfigResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetAccountTemporalConfigRequest + */ +export class SetAccountTemporalConfigRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 2; + */ + config?: AccountTemporalConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetAccountTemporalConfigRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "config", kind: "message", T: AccountTemporalConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountTemporalConfigRequest { + return new SetAccountTemporalConfigRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountTemporalConfigRequest { + return new SetAccountTemporalConfigRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetAccountTemporalConfigRequest { + return new SetAccountTemporalConfigRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetAccountTemporalConfigRequest | PlainMessage | undefined, b: SetAccountTemporalConfigRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetAccountTemporalConfigRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetAccountTemporalConfigResponse + */ +export class SetAccountTemporalConfigResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountTemporalConfig config = 1; + */ + config?: AccountTemporalConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetAccountTemporalConfigResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "config", kind: "message", T: AccountTemporalConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountTemporalConfigResponse { + return new SetAccountTemporalConfigResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountTemporalConfigResponse { + return new SetAccountTemporalConfigResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetAccountTemporalConfigResponse { + return new SetAccountTemporalConfigResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetAccountTemporalConfigResponse | PlainMessage | undefined, b: SetAccountTemporalConfigResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetAccountTemporalConfigResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AccountTemporalConfig + */ +export class AccountTemporalConfig extends Message { + /** + * @generated from field: string url = 1; + */ + url = ""; + + /** + * @generated from field: string namespace = 2; + */ + namespace = ""; + + /** + * @generated from field: string sync_job_queue_name = 3; + */ + syncJobQueueName = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccountTemporalConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "url", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "namespace", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "sync_job_queue_name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccountTemporalConfig { + return new AccountTemporalConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccountTemporalConfig { + return new AccountTemporalConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccountTemporalConfig { + return new AccountTemporalConfig().fromJsonString(jsonString, options); + } + + static equals(a: AccountTemporalConfig | PlainMessage | undefined, b: AccountTemporalConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(AccountTemporalConfig, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateTeamAccountRequest + */ +export class CreateTeamAccountRequest extends Message { + /** + * @generated from field: string name = 1; + */ + name = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateTeamAccountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateTeamAccountRequest { + return new CreateTeamAccountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateTeamAccountRequest { + return new CreateTeamAccountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateTeamAccountRequest { + return new CreateTeamAccountRequest().fromJsonString(jsonString, options); + } + + static equals(a: CreateTeamAccountRequest | PlainMessage | undefined, b: CreateTeamAccountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateTeamAccountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.CreateTeamAccountResponse + */ +export class CreateTeamAccountResponse extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.CreateTeamAccountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): CreateTeamAccountResponse { + return new CreateTeamAccountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): CreateTeamAccountResponse { + return new CreateTeamAccountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): CreateTeamAccountResponse { + return new CreateTeamAccountResponse().fromJsonString(jsonString, options); + } + + static equals(a: CreateTeamAccountResponse | PlainMessage | undefined, b: CreateTeamAccountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(CreateTeamAccountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AccountUser + */ +export class AccountUser extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string name = 2; + */ + name = ""; + + /** + * @generated from field: string image = 3; + */ + image = ""; + + /** + * @generated from field: string email = 4; + */ + email = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccountUser"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "name", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "image", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccountUser { + return new AccountUser().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccountUser { + return new AccountUser().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccountUser { + return new AccountUser().fromJsonString(jsonString, options); + } + + static equals(a: AccountUser | PlainMessage | undefined, b: AccountUser | PlainMessage | undefined): boolean { + return proto3.util.equals(AccountUser, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetTeamAccountMembersRequest + */ +export class GetTeamAccountMembersRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetTeamAccountMembersRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountMembersRequest { + return new GetTeamAccountMembersRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountMembersRequest { + return new GetTeamAccountMembersRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountMembersRequest { + return new GetTeamAccountMembersRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetTeamAccountMembersRequest | PlainMessage | undefined, b: GetTeamAccountMembersRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetTeamAccountMembersRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetTeamAccountMembersResponse + */ +export class GetTeamAccountMembersResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.AccountUser users = 1; + */ + users: AccountUser[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetTeamAccountMembersResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "users", kind: "message", T: AccountUser, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountMembersResponse { + return new GetTeamAccountMembersResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountMembersResponse { + return new GetTeamAccountMembersResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountMembersResponse { + return new GetTeamAccountMembersResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetTeamAccountMembersResponse | PlainMessage | undefined, b: GetTeamAccountMembersResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetTeamAccountMembersResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RemoveTeamAccountMemberRequest + */ +export class RemoveTeamAccountMemberRequest extends Message { + /** + * @generated from field: string user_id = 1; + */ + userId = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountMemberRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountMemberRequest { + return new RemoveTeamAccountMemberRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountMemberRequest { + return new RemoveTeamAccountMemberRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountMemberRequest { + return new RemoveTeamAccountMemberRequest().fromJsonString(jsonString, options); + } + + static equals(a: RemoveTeamAccountMemberRequest | PlainMessage | undefined, b: RemoveTeamAccountMemberRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(RemoveTeamAccountMemberRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RemoveTeamAccountMemberResponse + */ +export class RemoveTeamAccountMemberResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountMemberResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountMemberResponse { + return new RemoveTeamAccountMemberResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountMemberResponse { + return new RemoveTeamAccountMemberResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountMemberResponse { + return new RemoveTeamAccountMemberResponse().fromJsonString(jsonString, options); + } + + static equals(a: RemoveTeamAccountMemberResponse | PlainMessage | undefined, b: RemoveTeamAccountMemberResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(RemoveTeamAccountMemberResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.InviteUserToTeamAccountRequest + */ +export class InviteUserToTeamAccountRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: string email = 2; + */ + email = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.InviteUserToTeamAccountRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): InviteUserToTeamAccountRequest { + return new InviteUserToTeamAccountRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): InviteUserToTeamAccountRequest { + return new InviteUserToTeamAccountRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): InviteUserToTeamAccountRequest { + return new InviteUserToTeamAccountRequest().fromJsonString(jsonString, options); + } + + static equals(a: InviteUserToTeamAccountRequest | PlainMessage | undefined, b: InviteUserToTeamAccountRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(InviteUserToTeamAccountRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AccountInvite + */ +export class AccountInvite extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + /** + * @generated from field: string account_id = 2; + */ + accountId = ""; + + /** + * @generated from field: string sender_user_id = 3; + */ + senderUserId = ""; + + /** + * @generated from field: string email = 4; + */ + email = ""; + + /** + * @generated from field: string token = 5; + */ + token = ""; + + /** + * @generated from field: bool accepted = 6; + */ + accepted = false; + + /** + * @generated from field: google.protobuf.Timestamp created_at = 7; + */ + createdAt?: Timestamp; + + /** + * @generated from field: google.protobuf.Timestamp updated_at = 8; + */ + updatedAt?: Timestamp; + + /** + * @generated from field: google.protobuf.Timestamp expires_at = 9; + */ + expiresAt?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccountInvite"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "sender_user_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "email", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 6, name: "accepted", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 7, name: "created_at", kind: "message", T: Timestamp }, + { no: 8, name: "updated_at", kind: "message", T: Timestamp }, + { no: 9, name: "expires_at", kind: "message", T: Timestamp }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccountInvite { + return new AccountInvite().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccountInvite { + return new AccountInvite().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccountInvite { + return new AccountInvite().fromJsonString(jsonString, options); + } + + static equals(a: AccountInvite | PlainMessage | undefined, b: AccountInvite | PlainMessage | undefined): boolean { + return proto3.util.equals(AccountInvite, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.InviteUserToTeamAccountResponse + */ +export class InviteUserToTeamAccountResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountInvite invite = 1; + */ + invite?: AccountInvite; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.InviteUserToTeamAccountResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "invite", kind: "message", T: AccountInvite }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): InviteUserToTeamAccountResponse { + return new InviteUserToTeamAccountResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): InviteUserToTeamAccountResponse { + return new InviteUserToTeamAccountResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): InviteUserToTeamAccountResponse { + return new InviteUserToTeamAccountResponse().fromJsonString(jsonString, options); + } + + static equals(a: InviteUserToTeamAccountResponse | PlainMessage | undefined, b: InviteUserToTeamAccountResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(InviteUserToTeamAccountResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetTeamAccountInvitesRequest + */ +export class GetTeamAccountInvitesRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetTeamAccountInvitesRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountInvitesRequest { + return new GetTeamAccountInvitesRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountInvitesRequest { + return new GetTeamAccountInvitesRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountInvitesRequest { + return new GetTeamAccountInvitesRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetTeamAccountInvitesRequest | PlainMessage | undefined, b: GetTeamAccountInvitesRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetTeamAccountInvitesRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetTeamAccountInvitesResponse + */ +export class GetTeamAccountInvitesResponse extends Message { + /** + * @generated from field: repeated mgmt.v1alpha1.AccountInvite invites = 1; + */ + invites: AccountInvite[] = []; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetTeamAccountInvitesResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "invites", kind: "message", T: AccountInvite, repeated: true }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetTeamAccountInvitesResponse { + return new GetTeamAccountInvitesResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetTeamAccountInvitesResponse { + return new GetTeamAccountInvitesResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetTeamAccountInvitesResponse { + return new GetTeamAccountInvitesResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetTeamAccountInvitesResponse | PlainMessage | undefined, b: GetTeamAccountInvitesResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetTeamAccountInvitesResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RemoveTeamAccountInviteRequest + */ +export class RemoveTeamAccountInviteRequest extends Message { + /** + * @generated from field: string id = 1; + */ + id = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountInviteRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountInviteRequest { + return new RemoveTeamAccountInviteRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountInviteRequest { + return new RemoveTeamAccountInviteRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountInviteRequest { + return new RemoveTeamAccountInviteRequest().fromJsonString(jsonString, options); + } + + static equals(a: RemoveTeamAccountInviteRequest | PlainMessage | undefined, b: RemoveTeamAccountInviteRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(RemoveTeamAccountInviteRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.RemoveTeamAccountInviteResponse + */ +export class RemoveTeamAccountInviteResponse extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.RemoveTeamAccountInviteResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): RemoveTeamAccountInviteResponse { + return new RemoveTeamAccountInviteResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): RemoveTeamAccountInviteResponse { + return new RemoveTeamAccountInviteResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): RemoveTeamAccountInviteResponse { + return new RemoveTeamAccountInviteResponse().fromJsonString(jsonString, options); + } + + static equals(a: RemoveTeamAccountInviteResponse | PlainMessage | undefined, b: RemoveTeamAccountInviteResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(RemoveTeamAccountInviteResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AcceptTeamAccountInviteRequest + */ +export class AcceptTeamAccountInviteRequest extends Message { + /** + * @generated from field: string token = 1; + */ + token = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AcceptTeamAccountInviteRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "token", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AcceptTeamAccountInviteRequest { + return new AcceptTeamAccountInviteRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AcceptTeamAccountInviteRequest { + return new AcceptTeamAccountInviteRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AcceptTeamAccountInviteRequest { + return new AcceptTeamAccountInviteRequest().fromJsonString(jsonString, options); + } + + static equals(a: AcceptTeamAccountInviteRequest | PlainMessage | undefined, b: AcceptTeamAccountInviteRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(AcceptTeamAccountInviteRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AcceptTeamAccountInviteResponse + */ +export class AcceptTeamAccountInviteResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.UserAccount account = 1; + */ + account?: UserAccount; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AcceptTeamAccountInviteResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account", kind: "message", T: UserAccount }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AcceptTeamAccountInviteResponse { + return new AcceptTeamAccountInviteResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AcceptTeamAccountInviteResponse { + return new AcceptTeamAccountInviteResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AcceptTeamAccountInviteResponse { + return new AcceptTeamAccountInviteResponse().fromJsonString(jsonString, options); + } + + static equals(a: AcceptTeamAccountInviteResponse | PlainMessage | undefined, b: AcceptTeamAccountInviteResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(AcceptTeamAccountInviteResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetSystemInformationRequest + */ +export class GetSystemInformationRequest extends Message { + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemInformationRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemInformationRequest { + return new GetSystemInformationRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemInformationRequest { + return new GetSystemInformationRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemInformationRequest { + return new GetSystemInformationRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemInformationRequest | PlainMessage | undefined, b: GetSystemInformationRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemInformationRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetSystemInformationResponse + */ +export class GetSystemInformationResponse extends Message { + /** + * @generated from field: string version = 1; + */ + version = ""; + + /** + * @generated from field: string commit = 2; + */ + commit = ""; + + /** + * @generated from field: string compiler = 3; + */ + compiler = ""; + + /** + * @generated from field: string platform = 4; + */ + platform = ""; + + /** + * @generated from field: google.protobuf.Timestamp build_date = 5; + */ + buildDate?: Timestamp; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetSystemInformationResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "version", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "commit", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "compiler", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 4, name: "platform", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "build_date", kind: "message", T: Timestamp }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetSystemInformationResponse { + return new GetSystemInformationResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetSystemInformationResponse { + return new GetSystemInformationResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetSystemInformationResponse { + return new GetSystemInformationResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetSystemInformationResponse | PlainMessage | undefined, b: GetSystemInformationResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetSystemInformationResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountOnboardingConfigRequest + */ +export class GetAccountOnboardingConfigRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountOnboardingConfigRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountOnboardingConfigRequest { + return new GetAccountOnboardingConfigRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountOnboardingConfigRequest { + return new GetAccountOnboardingConfigRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountOnboardingConfigRequest { + return new GetAccountOnboardingConfigRequest().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountOnboardingConfigRequest | PlainMessage | undefined, b: GetAccountOnboardingConfigRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountOnboardingConfigRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.GetAccountOnboardingConfigResponse + */ +export class GetAccountOnboardingConfigResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 1; + */ + config?: AccountOnboardingConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.GetAccountOnboardingConfigResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "config", kind: "message", T: AccountOnboardingConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): GetAccountOnboardingConfigResponse { + return new GetAccountOnboardingConfigResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): GetAccountOnboardingConfigResponse { + return new GetAccountOnboardingConfigResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): GetAccountOnboardingConfigResponse { + return new GetAccountOnboardingConfigResponse().fromJsonString(jsonString, options); + } + + static equals(a: GetAccountOnboardingConfigResponse | PlainMessage | undefined, b: GetAccountOnboardingConfigResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(GetAccountOnboardingConfigResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetAccountOnboardingConfigRequest + */ +export class SetAccountOnboardingConfigRequest extends Message { + /** + * @generated from field: string account_id = 1; + */ + accountId = ""; + + /** + * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 2; + */ + config?: AccountOnboardingConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetAccountOnboardingConfigRequest"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "account_id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "config", kind: "message", T: AccountOnboardingConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountOnboardingConfigRequest { + return new SetAccountOnboardingConfigRequest().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountOnboardingConfigRequest { + return new SetAccountOnboardingConfigRequest().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetAccountOnboardingConfigRequest { + return new SetAccountOnboardingConfigRequest().fromJsonString(jsonString, options); + } + + static equals(a: SetAccountOnboardingConfigRequest | PlainMessage | undefined, b: SetAccountOnboardingConfigRequest | PlainMessage | undefined): boolean { + return proto3.util.equals(SetAccountOnboardingConfigRequest, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.SetAccountOnboardingConfigResponse + */ +export class SetAccountOnboardingConfigResponse extends Message { + /** + * @generated from field: mgmt.v1alpha1.AccountOnboardingConfig config = 1; + */ + config?: AccountOnboardingConfig; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.SetAccountOnboardingConfigResponse"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "config", kind: "message", T: AccountOnboardingConfig }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SetAccountOnboardingConfigResponse { + return new SetAccountOnboardingConfigResponse().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SetAccountOnboardingConfigResponse { + return new SetAccountOnboardingConfigResponse().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SetAccountOnboardingConfigResponse { + return new SetAccountOnboardingConfigResponse().fromJsonString(jsonString, options); + } + + static equals(a: SetAccountOnboardingConfigResponse | PlainMessage | undefined, b: SetAccountOnboardingConfigResponse | PlainMessage | undefined): boolean { + return proto3.util.equals(SetAccountOnboardingConfigResponse, a, b); + } +} + +/** + * @generated from message mgmt.v1alpha1.AccountOnboardingConfig + */ +export class AccountOnboardingConfig extends Message { + /** + * @generated from field: bool has_created_source_connection = 1; + */ + hasCreatedSourceConnection = false; + + /** + * @generated from field: bool has_created_destination_connection = 2; + */ + hasCreatedDestinationConnection = false; + + /** + * @generated from field: bool has_created_job = 3; + */ + hasCreatedJob = false; + + /** + * @generated from field: bool has_invited_members = 4; + */ + hasInvitedMembers = false; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "mgmt.v1alpha1.AccountOnboardingConfig"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "has_created_source_connection", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 2, name: "has_created_destination_connection", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 3, name: "has_created_job", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + { no: 4, name: "has_invited_members", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): AccountOnboardingConfig { + return new AccountOnboardingConfig().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): AccountOnboardingConfig { + return new AccountOnboardingConfig().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): AccountOnboardingConfig { + return new AccountOnboardingConfig().fromJsonString(jsonString, options); + } + + static equals(a: AccountOnboardingConfig | PlainMessage | undefined, b: AccountOnboardingConfig | PlainMessage | undefined): boolean { + return proto3.util.equals(AccountOnboardingConfig, a, b); + } +} + From d5f24d47ece3504f511ce8e76496c44ac7cd3fff Mon Sep 17 00:00:00 2001 From: Alisha Date: Mon, 22 Apr 2024 16:13:04 -0700 Subject: [PATCH 11/12] gen with fk working --- worker/internal/benthos/config.go | 2 +- .../sql/input_generate_table_records.go | 70 ++++++++++++++----- .../sql/input_generate_table_records_test.go | 32 +++++++++ worker/internal/benthos/sql/input_sql_raw.go | 2 +- .../benthos/sql/input_sql_select_generate.go | 4 +- .../gen-benthos-configs/benthos-builder.go | 6 +- .../gen-benthos-configs/generate-builder.go | 11 ++- .../gen-benthos-configs/processors.go | 2 +- .../datasync/activities/sync/activity.go | 4 ++ 9 files changed, 105 insertions(+), 28 deletions(-) diff --git a/worker/internal/benthos/config.go b/worker/internal/benthos/config.go index e603fec7ba..ecc0c081a3 100644 --- a/worker/internal/benthos/config.go +++ b/worker/internal/benthos/config.go @@ -61,7 +61,7 @@ type Inputs struct { } type GenerateSqlSelect struct { - Mapping string `json:"mapping" yaml:"mapping"` + Mapping string `json:"mapping,omitempty" yaml:"mapping,omitempty"` Count int `json:"count" yaml:"count"` Driver string `json:"driver" yaml:"driver"` Dsn string `json:"dsn" yaml:"dsn"` diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index 02d61a2086..b998266db1 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -31,7 +31,7 @@ func generateTableRecordsInputSpec() *service.ConfigSpec { func RegisterGenerateTableRecordsInput(env *service.Environment, dbprovider DbPoolProvider, stopActivityChannel chan error) error { return env.RegisterBatchInput( - "generate_table_records", generateTableRecordsInputSpec(), + "generate_sql_select", generateTableRecordsInputSpec(), func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { b, err := newGenerateReaderFromParsed(conf, mgr, dbprovider, stopActivityChannel) if err != nil { @@ -173,6 +173,7 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s } table := tables[0] + // need to remove self circular dependent tables otherTables := tables[1:] cols := s.tableColsMap[table] @@ -196,7 +197,7 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s rows, err := s.db.QueryContext(ctx, selectSql) if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { + if !neosync_benthos.ShouldTerminate(err.Error()) { s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) s.stopActivityChannel <- err } @@ -218,58 +219,84 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s otherTableRows := [][]map[string]any{} for _, t := range otherTables { cols := s.tableColsMap[t] + jsonF, _ := json.MarshalIndent(cols, "", " ") + fmt.Printf("\n cols: %s \n", string(jsonF)) selectColumns := make([]any, len(cols)) for i, col := range cols { - as, ok := s.columnNameMap[fmt.Sprintf("%s.%s", t, col)] + tn := fmt.Sprintf("%s.%s", t, col) + fmt.Println(tn) + as, ok := s.columnNameMap[tn] if ok { selectColumns[i] = goqu.I(col).As(as) } else { selectColumns[i] = col } } - selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomSmLimit)) + selectBuilder := builder.From(t).Select(selectColumns...).Order(orderBy).Limit(uint(randomSmLimit)) selectSql, _, err := selectBuilder.ToSQL() if err != nil { return nil, nil, err } + fmt.Printf("\n gen batched select sql additional: %s \n", selectSql) + rows, err := s.db.QueryContext(ctx, selectSql) if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { + if !neosync_benthos.ShouldTerminate(err.Error()) { s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) s.stopActivityChannel <- err } return nil, nil, err } - rowObjList, err := sqlRowsToMapList(rows) + rowObjList2, err := sqlRowsToMapList(rows) if err != nil { _ = rows.Close() return nil, nil, err } - otherTableRows = append(otherTableRows, rowObjList) + if len(rowObjList2) != 0 { + otherTableRows = append(otherTableRows, rowObjList2) + } } combinedRows := combineRowLists(otherTableRows) - for _, cr := range combinedRows { - var args map[string]any - if s.mapping != nil { - var iargs any - if iargs, err = s.mapping.Query(nil); err != nil { - return nil, nil, err + if len(combinedRows) > 0 { + for _, cr := range combinedRows { + var args map[string]any + if s.mapping != nil { + var iargs any + if iargs, err = s.mapping.Query(nil); err != nil { + return nil, nil, err + } + + var ok bool + if args, ok = iargs.(map[string]any); !ok { + err = fmt.Errorf("mapping returned non-array result: %T", iargs) + return nil, nil, err + } } + newRow := combineRows([]map[string]any{r, cr, args}) - var ok bool - if args, ok = iargs.(map[string]any); !ok { - err = fmt.Errorf("mapping returned non-array result: %T", iargs) + rowStr, err := json.Marshal(newRow) + if err != nil { return nil, nil, err } - } - newRow := combineRows([]map[string]any{r, cr, args}) + if s.remaining < 1 { + return batch, func(context.Context, error) error { return nil }, nil + } - rowStr, err := json.Marshal(newRow) + msg := service.NewMessage(rowStr) + batch = append(batch, msg) + s.remaining-- + } + } else { + rowStr, err := json.Marshal(r) if err != nil { return nil, nil, err } + if s.remaining < 1 { + return batch, func(context.Context, error) error { return nil }, nil + } msg := service.NewMessage(rowStr) batch = append(batch, msg) + s.remaining-- } } @@ -315,6 +342,11 @@ func combineRows(maps []map[string]any) map[string]any { } func combineRowLists(rows [][]map[string]any) []map[string]any { + jsonF, _ := json.MarshalIndent(rows, "", " ") + fmt.Printf("\n %s \n", string(jsonF)) + if len(rows) == 0 { + return []map[string]any{} + } results := []map[string]any{} rowCount := len(rows[0]) for i := 0; i < rowCount; i++ { diff --git a/worker/internal/benthos/sql/input_generate_table_records_test.go b/worker/internal/benthos/sql/input_generate_table_records_test.go index a5077dff65..1ead48fa47 100644 --- a/worker/internal/benthos/sql/input_generate_table_records_test.go +++ b/worker/internal/benthos/sql/input_generate_table_records_test.go @@ -49,3 +49,35 @@ func Test_combineRows(t *testing.T) { }) } } + +func TestCombineRowLists(t *testing.T) { + // Test case 1: Empty input + emptyResult := combineRowLists([][]map[string]any{}) + assert.Empty(t, emptyResult) + + // Test case 2: Single row with single map + singleRowSingleMap := [][]map[string]any{{{"a": 1, "b": 2}}} + singleResult := combineRowLists(singleRowSingleMap) + expectedSingleResult := []map[string]any{{"a": 1, "b": 2}} + assert.Equal(t, expectedSingleResult, singleResult) + + // Test case 3: Single row with multiple maps + singleRowMultipleMaps := [][]map[string]any{{{"a": 1}}, {{"b": 2}}, {{"c": 3}}} + multipleResult := combineRowLists(singleRowMultipleMaps) + expectedMultipleResult := []map[string]any{{"a": 1, "b": 2, "c": 3}} + assert.Equal(t, expectedMultipleResult, multipleResult) + + // Test case 4: Multiple rows with multiple maps + multipleRowsMultipleMaps := [][]map[string]any{ + {{"a": 1}, {"b": 2}, {"c": 3}}, + {{"d": 4}, {"e": 5}, {"f": 6}}, + {{"g": 7}, {"h": 8}, {"i": 9}}, + } + multipleRowsResult := combineRowLists(multipleRowsMultipleMaps) + expectedMultipleRowsResult := []map[string]any{ + {"a": 1, "d": 4, "g": 7}, + {"b": 2, "e": 5, "h": 8}, + {"c": 3, "f": 6, "i": 9}, + } + assert.Equal(t, expectedMultipleRowsResult, multipleRowsResult) +} diff --git a/worker/internal/benthos/sql/input_sql_raw.go b/worker/internal/benthos/sql/input_sql_raw.go index ec8553e10b..4f1632abd6 100644 --- a/worker/internal/benthos/sql/input_sql_raw.go +++ b/worker/internal/benthos/sql/input_sql_raw.go @@ -119,7 +119,7 @@ func (s *pooledInput) Connect(ctx context.Context) error { rows, err := db.QueryContext(ctx, s.queryStatic, args...) if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { + if !neosync_benthos.ShouldTerminate(err.Error()) { s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) s.stopActivityChannel <- err } diff --git a/worker/internal/benthos/sql/input_sql_select_generate.go b/worker/internal/benthos/sql/input_sql_select_generate.go index c9b7d6fba9..669deaf5d3 100644 --- a/worker/internal/benthos/sql/input_sql_select_generate.go +++ b/worker/internal/benthos/sql/input_sql_select_generate.go @@ -175,7 +175,7 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { rows, err := db.QueryContext(ctx, selectSql) if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { + if !neosync_benthos.ShouldTerminate(err.Error()) { s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) s.stopActivityChannel <- err } @@ -207,7 +207,7 @@ func (s *sqlSelectGenerateInput) Connect(ctx context.Context) error { } rows, err := db.QueryContext(ctx, selectSql) if err != nil { - if !neosync_benthos.IsMaxConnectionError(err.Error()) { + if !neosync_benthos.ShouldTerminate(err.Error()) { s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) s.stopActivityChannel <- err } diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go index 78d95c2b3d..9a97a5196c 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/benthos-builder.go @@ -356,7 +356,7 @@ func (b *benthosBuilder) GenerateBenthosConfigs( } updateResponses = append(updateResponses, updateResp) } - } else if resp.Config.Input.Generate != nil { + } else if resp.Config.Input.Generate != nil || resp.Config.Input.GenerateSqlSelect != nil { cols := buildPlainColumns(tm.Mappings) // processorConfigs := []neosync_benthos.ProcessorConfig{} // for _, pc := range resp.Processors { @@ -1130,12 +1130,16 @@ func getSqlJobSourceOpts( } func (b *benthosBuilder) getJobSourceConnection(ctx context.Context, jobSource *mgmtv1alpha1.JobSource) (*mgmtv1alpha1.Connection, error) { + jsonF, _ := json.MarshalIndent(jobSource, "", " ") + fmt.Printf("\n %s \n", string(jsonF)) var connectionId string switch jobSourceConfig := jobSource.GetOptions().GetConfig().(type) { case *mgmtv1alpha1.JobSourceOptions_Postgres: connectionId = jobSourceConfig.Postgres.GetConnectionId() case *mgmtv1alpha1.JobSourceOptions_Mysql: connectionId = jobSourceConfig.Mysql.GetConnectionId() + case *mgmtv1alpha1.JobSourceOptions_Generate: + connectionId = jobSourceConfig.Generate.GetFkSourceConnectionId() default: return nil, errors.New("unsupported job source options type") } diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go index 890dc3fbb2..32a5bc7199 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go @@ -74,6 +74,10 @@ func buildBenthosGenerateSourceConfigResponses( } var bc *neosync_benthos.BenthosConfig + cols := []string{} + for _, m := range tableMapping.Mappings { + cols = append(cols, m.Column) + } if len(runConfigs) > 0 && len(runConfigs[0].DependsOn) > 0 { columnNameMap := map[string]string{} tableColsMaps := map[string][]string{} @@ -81,7 +85,7 @@ func buildBenthosGenerateSourceConfigResponses( constraints := tableConstraintsMap[tableName] for _, tc := range constraints.Constraints { columnNameMap[fmt.Sprintf("%s.%s", tc.ForeignKey.Table, tc.ForeignKey.Column)] = tc.Column - tableColsMaps[tc.ForeignKey.Table] = append(tableColsMaps[tc.ForeignKey.Table], tc.ForeignKey.Table) + tableColsMaps[tc.ForeignKey.Table] = append(tableColsMaps[tc.ForeignKey.Table], tc.ForeignKey.Column) } bc = &neosync_benthos.BenthosConfig{ @@ -89,8 +93,8 @@ func buildBenthosGenerateSourceConfigResponses( Input: &neosync_benthos.InputConfig{ Inputs: neosync_benthos.Inputs{ GenerateSqlSelect: &neosync_benthos.GenerateSqlSelect{ - Count: count, - Mapping: mutations, + Count: count, + // Mapping: mutations, Driver: driver, Dsn: "${SOURCE_CONNECTION_DSN}", TableColumnsMap: tableColsMaps, @@ -148,6 +152,7 @@ func buildBenthosGenerateSourceConfigResponses( TableSchema: tableMapping.Schema, TableName: tableMapping.Table, + Columns: cols, // Processors: processors, diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go index 551e3a96a4..8662cfcfcd 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/processors.go @@ -114,7 +114,6 @@ func buildMutationConfigs( mutations := []string{} for _, col := range cols { - colInfo := tableColumnInfo[col.Column] if shouldProcessColumn(col.Transformer) { if _, ok := col.Transformer.Config.Config.(*mgmtv1alpha1.TransformerConfig_UserDefinedTransformerConfig); ok { // handle user defined transformer -> get the user defined transformer configs using the id @@ -125,6 +124,7 @@ func buildMutationConfigs( col.Transformer = val } if col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_TRANSFORM_JAVASCRIPT && col.Transformer.Source != mgmtv1alpha1.TransformerSource_TRANSFORMER_SOURCE_GENERATE_JAVASCRIPT { + colInfo := tableColumnInfo[col.Column] mutation, err := computeMutationFunction(col, colInfo) if err != nil { return "", fmt.Errorf("%s is not a supported transformer: %w", col.Transformer, err) diff --git a/worker/pkg/workflows/datasync/activities/sync/activity.go b/worker/pkg/workflows/datasync/activities/sync/activity.go index d9af93cb90..9586e23a26 100644 --- a/worker/pkg/workflows/datasync/activities/sync/activity.go +++ b/worker/pkg/workflows/datasync/activities/sync/activity.go @@ -245,6 +245,10 @@ func (a *Activity) Sync(ctx context.Context, req *SyncRequest, metadata *SyncMet if err != nil { return nil, fmt.Errorf("unable to register pooled_sql_raw input to benthos instance: %w", err) } + err = neosync_benthos_sql.RegisterGenerateTableRecordsInput(benthosenv, poolprovider, stopActivityChan) + if err != nil { + return nil, fmt.Errorf("unable to register generate_sql_select input to benthos instance: %w", err) + } err = neosync_benthos_error.RegisterErrorProcessor(benthosenv, stopActivityChan) if err != nil { From fa7f3922257bf3721193041b767b8ccc518e9fdf Mon Sep 17 00:00:00 2001 From: Alisha Date: Tue, 23 Apr 2024 09:27:22 -0700 Subject: [PATCH 12/12] stash --- .../sql/input_generate_table_records.go | 115 ++++++++++-------- .../gen-benthos-configs/generate-builder.go | 9 +- 2 files changed, 66 insertions(+), 58 deletions(-) diff --git a/worker/internal/benthos/sql/input_generate_table_records.go b/worker/internal/benthos/sql/input_generate_table_records.go index b998266db1..518c79cd90 100644 --- a/worker/internal/benthos/sql/input_generate_table_records.go +++ b/worker/internal/benthos/sql/input_generate_table_records.go @@ -187,31 +187,13 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s selectColumns[i] = col } } - orderBy := exp.NewOrderedExpression(exp.NewLiteralExpression(sqlRandomStr), exp.AscDir, exp.NullsLastSortType) - builder := goqu.Dialect(s.driver) - selectBuilder := builder.From(table).Select(selectColumns...).Order(orderBy).Limit(uint(randomLrgLimit)) - selectSql, _, err := selectBuilder.ToSQL() + rows, err := s.queryDatabase(sqlRandomStr, table, randomLrgLimit, selectColumns) if err != nil { return nil, nil, err } - rows, err := s.db.QueryContext(ctx, selectSql) - if err != nil { - if !neosync_benthos.ShouldTerminate(err.Error()) { - s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) - s.stopActivityChannel <- err - } - return nil, nil, err - } - - rowObjList, err := sqlRowsToMapList(rows) - if err != nil { - _ = rows.Close() - return nil, nil, err - } - batch := service.MessageBatch{} - for _, r := range rowObjList { + for _, r := range rows { randomSmLimit, err := transformer_utils.GenerateRandomInt64InValueRange(0, 3) if err != nil { return nil, nil, err @@ -219,12 +201,9 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s otherTableRows := [][]map[string]any{} for _, t := range otherTables { cols := s.tableColsMap[t] - jsonF, _ := json.MarshalIndent(cols, "", " ") - fmt.Printf("\n cols: %s \n", string(jsonF)) selectColumns := make([]any, len(cols)) for i, col := range cols { tn := fmt.Sprintf("%s.%s", t, col) - fmt.Println(tn) as, ok := s.columnNameMap[tn] if ok { selectColumns[i] = goqu.I(col).As(as) @@ -232,28 +211,13 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s selectColumns[i] = col } } - selectBuilder := builder.From(t).Select(selectColumns...).Order(orderBy).Limit(uint(randomSmLimit)) - selectSql, _, err := selectBuilder.ToSQL() - if err != nil { - return nil, nil, err - } - fmt.Printf("\n gen batched select sql additional: %s \n", selectSql) - - rows, err := s.db.QueryContext(ctx, selectSql) - if err != nil { - if !neosync_benthos.ShouldTerminate(err.Error()) { - s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) - s.stopActivityChannel <- err - } - return nil, nil, err - } - rowObjList2, err := sqlRowsToMapList(rows) + newRows, err := s.queryDatabase(sqlRandomStr, t, randomSmLimit, selectColumns) if err != nil { - _ = rows.Close() return nil, nil, err } - if len(rowObjList2) != 0 { - otherTableRows = append(otherTableRows, rowObjList2) + if len(newRows) != 0 { + // how to handle tables that don't have enough data + otherTableRows = append(otherTableRows, newRows) } } combinedRows := combineRowLists(otherTableRows) @@ -261,19 +225,13 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s for _, cr := range combinedRows { var args map[string]any if s.mapping != nil { - var iargs any - if iargs, err = s.mapping.Query(nil); err != nil { - return nil, nil, err - } - - var ok bool - if args, ok = iargs.(map[string]any); !ok { - err = fmt.Errorf("mapping returned non-array result: %T", iargs) + args, err = s.queryBloblangMapping() + if err != nil { return nil, nil, err } } - newRow := combineRows([]map[string]any{r, cr, args}) + newRow := combineRows([]map[string]any{r, cr, args}) rowStr, err := json.Marshal(newRow) if err != nil { return nil, nil, err @@ -287,7 +245,16 @@ func (s *generateReader) ReadBatch(ctx context.Context) (service.MessageBatch, s s.remaining-- } } else { - rowStr, err := json.Marshal(r) + newRow := r + if s.mapping != nil { + args, err := s.queryBloblangMapping() + if err != nil { + return nil, nil, err + } + newRow = combineRows([]map[string]any{r, args}) + } + + rowStr, err := json.Marshal(newRow) if err != nil { return nil, nil, err } @@ -319,6 +286,48 @@ func (s *generateReader) Close(ctx context.Context) (err error) { return nil } +func (s *generateReader) queryBloblangMapping() (map[string]any, error) { + var iargs any + var err error + if iargs, err = s.mapping.Query(nil); err != nil { + return nil, err + } + + var ok bool + var args map[string]any + if args, ok = iargs.(map[string]any); !ok { + err = fmt.Errorf("mapping returned non-array result: %T", iargs) + return nil, err + } + return args, nil +} + +func (s *generateReader) queryDatabase(sqlRandomStr, table string, limit int64, columns []any) ([]map[string]any, error) { + orderBy := exp.NewOrderedExpression(exp.NewLiteralExpression(sqlRandomStr), exp.AscDir, exp.NullsLastSortType) + builder := goqu.Dialect(s.driver) + selectBuilder := builder.From(table).Select(columns...).Order(orderBy).Limit(uint(limit)) + selectSql, _, err := selectBuilder.ToSQL() + if err != nil { + return nil, err + } + + rows, err := s.db.QueryContext(ctx, selectSql) + if err != nil { + if !neosync_benthos.ShouldTerminate(err.Error()) { + s.logger.Error(fmt.Sprintf("Benthos input error - sending stop activity signal: %s ", err.Error())) + s.stopActivityChannel <- err + } + return nil, err + } + + rowObjList, err := sqlRowsToMapList(rows) + if err != nil { + _ = rows.Close() + return nil, err + } + return rowObjList, nil +} + func sqlRowsToMapList(rows *sql.Rows) ([]map[string]any, error) { results := []map[string]any{} for rows.Next() { @@ -342,8 +351,6 @@ func combineRows(maps []map[string]any) map[string]any { } func combineRowLists(rows [][]map[string]any) []map[string]any { - jsonF, _ := json.MarshalIndent(rows, "", " ") - fmt.Printf("\n %s \n", string(jsonF)) if len(rows) == 0 { return []map[string]any{} } diff --git a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go index 32a5bc7199..ed3425167a 100644 --- a/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go +++ b/worker/pkg/workflows/datasync/activities/gen-benthos-configs/generate-builder.go @@ -93,8 +93,8 @@ func buildBenthosGenerateSourceConfigResponses( Input: &neosync_benthos.InputConfig{ Inputs: neosync_benthos.Inputs{ GenerateSqlSelect: &neosync_benthos.GenerateSqlSelect{ - Count: count, - // Mapping: mutations, + Count: count, + Mapping: mutations, Driver: driver, Dsn: "${SOURCE_CONNECTION_DSN}", TableColumnsMap: tableColsMaps, @@ -103,8 +103,9 @@ func buildBenthosGenerateSourceConfigResponses( }, }, Pipeline: &neosync_benthos.PipelineConfig{ - Threads: -1, - Processors: processors, + Threads: -1, + // Processors: processors, + Processors: []neosync_benthos.ProcessorConfig{}, }, Output: &neosync_benthos.OutputConfig{ Outputs: neosync_benthos.Outputs{