From 5d7d15aa04cf365b4e2eb3461bf03f43dbf2fe5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 7 Mar 2024 22:25:23 +0000 Subject: [PATCH 01/31] Yet another kafka connector This time with Lua scripting Need to implement flatbuffer library ourselves sicne string.pack/string.unpack are missing --- flow/connectors/core.go | 4 + flow/connectors/eventhub/eventhub.go | 12 +- flow/connectors/kafka/kafka.go | 227 +++++++ flow/connectors/postgres/postgres.go | 6 +- flow/go.mod | 4 + flow/go.sum | 8 + flow/model/model.go | 44 +- flow/model/record_items.go | 3 +- flow/pua/flatbuffers.go | 47 ++ flow/pua/flatbuffers_binaryarray.go | 129 ++++ flow/pua/flatbuffers_builder.go | 591 +++++++++++++++++++ flow/pua/flatbuffers_numtypes.go | 235 ++++++++ flow/pua/flatbuffers_view.go | 224 +++++++ flow/pua/peerdb.go | 202 +++++++ flow/pua/userdata.go | 41 ++ nexus/analyzer/src/lib.rs | 32 +- nexus/catalog/migrations/V21__scripts.sql | 8 + nexus/catalog/src/lib.rs | 6 + nexus/pt/src/lib.rs | 2 +- protos/peers.proto | 10 + ui/README.md | 2 +- ui/app/api/peers/getTruePeer.ts | 16 +- ui/app/api/peers/info/[peerName]/route.ts | 19 +- ui/app/peers/create/[peerType]/handlers.ts | 5 + ui/app/peers/create/[peerType]/helpers/ka.ts | 33 ++ ui/app/peers/create/[peerType]/page.tsx | 3 + ui/app/peers/create/[peerType]/schema.ts | 11 + ui/components/PeerComponent.tsx | 3 + ui/components/PeerForms/KafkaConfig.tsx | 77 +++ ui/components/PeerTypeComponent.tsx | 2 + ui/components/SelectSource.tsx | 3 +- ui/public/svgs/kafka.svg | 1 + 32 files changed, 1964 insertions(+), 46 deletions(-) create mode 100644 flow/connectors/kafka/kafka.go create mode 100644 flow/pua/flatbuffers.go create mode 100644 flow/pua/flatbuffers_binaryarray.go create mode 100644 flow/pua/flatbuffers_builder.go create mode 100644 flow/pua/flatbuffers_numtypes.go create mode 100644 flow/pua/flatbuffers_view.go create mode 100644 flow/pua/peerdb.go create mode 100644 flow/pua/userdata.go create mode 100644 nexus/catalog/migrations/V21__scripts.sql create mode 100644 ui/app/peers/create/[peerType]/helpers/ka.ts create mode 100644 ui/components/PeerForms/KafkaConfig.tsx create mode 100644 ui/public/svgs/kafka.svg diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 39e31f8171..4251c1726a 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -11,6 +11,7 @@ import ( connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connclickhouse "github.com/PeerDB-io/peer-flow/connectors/clickhouse" conneventhub "github.com/PeerDB-io/peer-flow/connectors/eventhub" + connkafka "github.com/PeerDB-io/peer-flow/connectors/kafka" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" conns3 "github.com/PeerDB-io/peer-flow/connectors/s3" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" @@ -202,6 +203,8 @@ func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { return connsqlserver.NewSQLServerConnector(ctx, inner.SqlserverConfig) case *protos.Peer_ClickhouseConfig: return connclickhouse.NewClickhouseConnector(ctx, inner.ClickhouseConfig) + case *protos.Peer_KafkaConfig: + return connkafka.NewKafkaConnector(ctx, inner.KafkaConfig) default: return nil, ErrUnsupportedFunctionality } @@ -260,6 +263,7 @@ var ( _ CDCSyncConnector = &connbigquery.BigQueryConnector{} _ CDCSyncConnector = &connsnowflake.SnowflakeConnector{} _ CDCSyncConnector = &conneventhub.EventHubConnector{} + _ CDCSyncConnector = &connkafka.KafkaConnector{} _ CDCSyncConnector = &conns3.S3Connector{} _ CDCSyncConnector = &connclickhouse.ClickhouseConnector{} diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index d34c1d9274..7d75298777 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -88,13 +88,7 @@ func (c *EventHubConnector) GetLastOffset(ctx context.Context, jobName string) ( } func (c *EventHubConnector) SetLastOffset(ctx context.Context, jobName string, offset int64) error { - err := c.pgMetadata.UpdateLastOffset(ctx, jobName, offset) - if err != nil { - c.logger.Error("failed to update last offset", slog.Any("error", err)) - return err - } - - return nil + return c.pgMetadata.UpdateLastOffset(ctx, jobName, offset) } // returns the number of records synced @@ -204,9 +198,7 @@ func (c *EventHubConnector) processBatch( } func (c *EventHubConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - batch := req.Records - - numRecords, err := c.processBatch(ctx, req.FlowJobName, batch) + numRecords, err := c.processBatch(ctx, req.FlowJobName, req.Records) if err != nil { c.logger.Error("failed to process batch", slog.Any("error", err)) return nil, err diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go new file mode 100644 index 0000000000..3ccb17d31f --- /dev/null +++ b/flow/connectors/kafka/kafka.go @@ -0,0 +1,227 @@ +package connkafka + +import ( + "bytes" + "context" + "crypto/tls" + "errors" + "fmt" + "log/slog" + "sync" + + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/sasl/scram" + "github.com/twmb/franz-go/plugin/kslog" + "github.com/yuin/gopher-lua" + "go.temporal.io/sdk/log" + + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/logger" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/peerdbenv" + "github.com/PeerDB-io/peer-flow/pua" +) + +type KafkaConnector struct { + client *kgo.Client + pgMetadata *metadataStore.PostgresMetadataStore + logger log.Logger +} + +func NewKafkaConnector( + ctx context.Context, + config *protos.KafkaConfig, +) (*KafkaConnector, error) { + optionalOpts := append( + make([]kgo.Opt, 0, 6), + kgo.SeedBrokers(config.Servers...), + kgo.AllowAutoTopicCreation(), + kgo.WithLogger(kslog.New(slog.Default())), // TODO use logger.LoggerFromCtx + kgo.SoftwareNameAndVersion("peerdb", peerdbenv.PeerDBVersionShaShort()), + ) + if !config.DisableTls { + optionalOpts = append(optionalOpts, kgo.DialTLSConfig(&tls.Config{MinVersion: tls.VersionTLS13})) + } + if config.Username != "" { + auth := scram.Auth{User: config.Username, Pass: config.Password} + switch config.Sasl { + case "SCRAM-SHA-256": + optionalOpts = append(optionalOpts, kgo.SASL(auth.AsSha256Mechanism())) + case "SCRAM-SHA-512": + optionalOpts = append(optionalOpts, kgo.SASL(auth.AsSha512Mechanism())) + default: + return nil, fmt.Errorf("unsupported SASL mechanism: %s", config.Sasl) + } + } + client, err := kgo.NewClient(optionalOpts...) + if err != nil { + return nil, fmt.Errorf("failed to create kafka client: %w", err) + } + + return &KafkaConnector{ + client: client, + logger: logger.LoggerFromCtx(ctx), + }, nil +} + +func (c *KafkaConnector) Close() error { + if c != nil { + c.client.Close() + } + return nil +} + +func (c *KafkaConnector) ConnectionActive(ctx context.Context) error { + return c.client.Ping(ctx) +} + +func (c *KafkaConnector) CreateRawTable(ctx context.Context, req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) { + return &protos.CreateRawTableOutput{TableIdentifier: "n/a"}, nil +} + +func (c *KafkaConnector) GetLastSyncBatchID(ctx context.Context, jobName string) (int64, error) { + return c.pgMetadata.GetLastBatchID(ctx, jobName) +} + +func (c *KafkaConnector) GetLastOffset(ctx context.Context, jobName string) (int64, error) { + return c.pgMetadata.FetchLastOffset(ctx, jobName) +} + +func (c *KafkaConnector) SetLastOffset(ctx context.Context, jobName string, offset int64) error { + return c.pgMetadata.UpdateLastOffset(ctx, jobName, offset) +} + +func (c *KafkaConnector) NeedsSetupMetadataTables(_ context.Context) bool { + return false +} + +func (c *KafkaConnector) SetupMetadataTables(_ context.Context) error { + return nil +} + +func (c *KafkaConnector) ReplayTableSchemaDeltas(_ context.Context, flowJobName string, schemaDeltas []*protos.TableSchemaDelta) error { + c.logger.Info("ReplayTableSchemaDeltas for event hub is a no-op") + return nil +} + +func (c *KafkaConnector) SyncFlowCleanup(ctx context.Context, jobName string) error { + return c.pgMetadata.DropMetadata(ctx, jobName) +} + +func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { + err := c.client.BeginTransaction() + if err != nil { + return nil, err + } + + var wg sync.WaitGroup + wgCtx, wgErr := context.WithCancelCause(ctx) + produceCb := func(r *kgo.Record, err error) { + if err != nil { + wgErr(err) + } + wg.Done() + } + + numRecords := int64(0) + tableNameRowsMapping := make(map[string]uint32) + + var fn *lua.LFunction + var ls *lua.LState + if req.Script != "" { + ls = lua.NewState(lua.Options{SkipOpenLibs: true}) + defer ls.Close() + ls.SetContext(wgCtx) + for _, pair := range []struct { + n string + f lua.LGFunction + }{ + {lua.LoadLibName, lua.OpenPackage}, // Must be first + {lua.BaseLibName, lua.OpenBase}, + {lua.TabLibName, lua.OpenTable}, + {lua.StringLibName, lua.OpenString}, + {lua.MathLibName, lua.OpenMath}, + } { + ls.Push(ls.NewFunction(pair.f)) + ls.Push(lua.LString(pair.n)) + err := ls.PCall(1, 0, nil) + if err != nil { + return nil, fmt.Errorf("failed to initialize Lua runtime: %w", err) + } + } + ls.PreloadModule("flatbuffers", pua.FlatBuffers_Loader) + pua.RegisterTypes(ls) + err := ls.DoString(req.Script) + if err != nil { + return nil, fmt.Errorf("error while executing script: %w", err) + } + + var ok bool + fn, ok = ls.GetGlobal("onRow").(*lua.LFunction) + if !ok { + return nil, errors.New("script should define `onRow` function") + } + } else { + return nil, errors.New("kafka mirror must have script") + } + + for record := range req.Records.GetRecords() { + if err := wgCtx.Err(); err != nil { + return nil, err + } + topic := record.GetDestinationTableName() + ls.Push(fn) + ls.Push(pua.LuaRecord.New(ls, record)) + err := ls.PCall(1, 1, nil) + if err != nil { + return nil, fmt.Errorf("script failed: %w", err) + } + value := ls.Get(-1) + if value != lua.LNil { + lstr, ok := value.(lua.LString) + if !ok { + return nil, fmt.Errorf("script returned non-nil non-string: %v", value) + } + wg.Add(1) + c.client.Produce(wgCtx, &kgo.Record{Topic: topic, Value: bytes.Clone([]byte(lstr))}, produceCb) + + numRecords += 1 + tableNameRowsMapping[topic] += 1 + } + } + + // TODO handle + waitChan := make(chan struct{}) + go func() { + wg.Wait() + waitChan <- struct{}{} + }() + select { + case <-wgCtx.Done(): + return nil, wgCtx.Err() + case <-waitChan: + } + + if err := c.client.Flush(ctx); err != nil { + return nil, fmt.Errorf("could not flush transaction: %w", err) + } + + if err := c.client.EndTransaction(ctx, kgo.TryCommit); err != nil { + return nil, fmt.Errorf("could not commit transaction: %w", err) + } + + lastCheckpoint := req.Records.GetLastCheckpoint() + err = c.pgMetadata.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) + if err != nil { + return nil, err + } + + return &model.SyncResponse{ + CurrentSyncBatchID: req.SyncBatchID, + LastSyncedCheckpointID: lastCheckpoint, + NumRecordsSynced: numRecords, + TableNameRowsMapping: tableNameRowsMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, + }, nil +} diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index bd6721c076..c57154bd7c 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -404,7 +404,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Postgres table %s via COPY", rawTableIdentifier)) - numRecords := 0 + numRecords := int64(0) tableNameRowsMapping := make(map[string]uint32) streamReadFunc := func() ([]any, error) { @@ -509,7 +509,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco if err != nil { return nil, fmt.Errorf("error syncing records: %w", err) } - if syncedRecordsCount != int64(numRecords) { + if syncedRecordsCount != numRecords { return nil, fmt.Errorf("error syncing records: expected %d records to be synced, but %d were synced", numRecords, syncedRecordsCount) } @@ -536,7 +536,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco return &model.SyncResponse{ LastSyncedCheckpointID: lastCP, - NumRecordsSynced: int64(numRecords), + NumRecordsSynced: numRecords, CurrentSyncBatchID: req.SyncBatchID, TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: req.Records.SchemaDeltas, diff --git a/flow/go.mod b/flow/go.mod index 154a068d20..433d8c912c 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -35,9 +35,12 @@ require ( github.com/slack-go/slack v0.12.5 github.com/snowflakedb/gosnowflake v1.8.0 github.com/stretchr/testify v1.9.0 + github.com/twmb/franz-go v1.16.1 + github.com/twmb/franz-go/plugin/kslog v1.0.0 github.com/twpayne/go-geos v0.17.0 github.com/urfave/cli/v3 v3.0.0-alpha9 github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a + github.com/yuin/gopher-lua v1.1.1 go.temporal.io/api v1.29.1 go.temporal.io/sdk v1.26.0 go.uber.org/automaxprocs v1.5.3 @@ -91,6 +94,7 @@ require ( github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect + github.com/twmb/franz-go/pkg/kmsg v1.7.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect go.opentelemetry.io/otel v1.24.0 // indirect diff --git a/flow/go.sum b/flow/go.sum index 377b2c2a16..4a3616a854 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -367,6 +367,12 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= +github.com/twmb/franz-go v1.16.1 h1:rpWc7fB9jd7TgmCyfxzenBI+QbgS8ZfJOUQE+tzPtbE= +github.com/twmb/franz-go v1.16.1/go.mod h1:/pER254UPPGp/4WfGqRi+SIRGE50RSQzVubQp6+N4FA= +github.com/twmb/franz-go/pkg/kmsg v1.7.0 h1:a457IbvezYfA5UkiBvyV3zj0Is3y1i8EJgqjJYoij2E= +github.com/twmb/franz-go/pkg/kmsg v1.7.0/go.mod h1:se9Mjdt0Nwzc9lnjJ0HyDtLyBnaBDAd7pCje47OhSyw= +github.com/twmb/franz-go/plugin/kslog v1.0.0 h1:I64oEmF+0PDvmyLgwrlOtg4mfpSE9GwlcLxM4af2t60= +github.com/twmb/franz-go/plugin/kslog v1.0.0/go.mod h1:8pMjK3OJJJNNYddBSbnXZkIK5dCKFIk9GcVVCDgvnQc= github.com/twpayne/go-geos v0.17.0 h1:158IwlZxA5Q1qWpBrP90dG3B8mQ5yb5RA4SPhR2CJ2E= github.com/twpayne/go-geos v0.17.0/go.mod h1:OgP9eXBQBbU1Qi2IR3kF608WTd9YtRXZP0FugQ0POi0= github.com/urfave/cli/v3 v3.0.0-alpha9 h1:P0RMy5fQm1AslQS+XCmy9UknDXctOmG/q/FZkUFnJSo= @@ -383,6 +389,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yuin/gopher-lua v1.1.1 h1:kYKnWBjvbNP4XLT3+bPEwAXJx262OhaHDWDVOPjL46M= +github.com/yuin/gopher-lua v1.1.1/go.mod h1:GBR0iDaNXjAgGg9zfCvksxSRnQx76gclCIb7kdAd1Pw= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= diff --git a/flow/model/model.go b/flow/model/model.go index 9e38e237de..06f56ec2f3 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -14,9 +14,12 @@ type NameAndExclude struct { } func NewNameAndExclude(name string, exclude []string) NameAndExclude { - exset := make(map[string]struct{}, len(exclude)) - for _, col := range exclude { - exset[col] = struct{}{} + var exset map[string]struct{} + if len(exclude) != 0 { + exset = make(map[string]struct{}, len(exclude)) + for _, col := range exclude { + exset[col] = struct{}{} + } } return NameAndExclude{Name: name, Exclude: exset} } @@ -45,10 +48,9 @@ type PullRecordsRequest struct { } type Record interface { - // GetCheckpointID returns the ID of the record. GetCheckpointID() int64 - // get table name GetDestinationTableName() string + GetSourceTableName() string // get columns and values for the record GetItems() *RecordItems } @@ -59,9 +61,12 @@ type ToJSONOptions struct { } func NewToJSONOptions(unnestCols []string, hstoreAsJSON bool) *ToJSONOptions { - unnestColumns := make(map[string]struct{}, len(unnestCols)) - for _, col := range unnestCols { - unnestColumns[col] = struct{}{} + var unnestColumns map[string]struct{} + if len(unnestCols) != 0 { + unnestColumns = make(map[string]struct{}, len(unnestCols)) + for _, col := range unnestCols { + unnestColumns[col] = struct{}{} + } } return &ToJSONOptions{ UnnestColumns: unnestColumns, @@ -82,7 +87,6 @@ type InsertRecord struct { Items *RecordItems } -// Implement Record interface for InsertRecord. func (r *InsertRecord) GetCheckpointID() int64 { return r.CheckpointID } @@ -91,6 +95,10 @@ func (r *InsertRecord) GetDestinationTableName() string { return r.DestinationTableName } +func (r *InsertRecord) GetSourceTableName() string { + return r.SourceTableName +} + func (r *InsertRecord) GetItems() *RecordItems { return r.Items } @@ -110,16 +118,18 @@ type UpdateRecord struct { UnchangedToastColumns map[string]struct{} } -// Implement Record interface for UpdateRecord. func (r *UpdateRecord) GetCheckpointID() int64 { return r.CheckpointID } -// Implement Record interface for UpdateRecord. func (r *UpdateRecord) GetDestinationTableName() string { return r.DestinationTableName } +func (r *UpdateRecord) GetSourceTableName() string { + return r.SourceTableName +} + func (r *UpdateRecord) GetItems() *RecordItems { return r.NewItems } @@ -137,7 +147,6 @@ type DeleteRecord struct { UnchangedToastColumns map[string]struct{} } -// Implement Record interface for DeleteRecord. func (r *DeleteRecord) GetCheckpointID() int64 { return r.CheckpointID } @@ -146,6 +155,10 @@ func (r *DeleteRecord) GetDestinationTableName() string { return r.DestinationTableName } +func (r *DeleteRecord) GetSourceTableName() string { + return r.SourceTableName +} + func (r *DeleteRecord) GetItems() *RecordItems { return r.Items } @@ -165,6 +178,8 @@ type SyncRecordsRequest struct { TableMappings []*protos.TableMapping // Staging path for AVRO files in CDC StagingPath string + // Lua script + Script string } type NormalizeRecordsRequest struct { @@ -208,7 +223,6 @@ type RelationRecord struct { TableSchemaDelta *protos.TableSchemaDelta `json:"tableSchemaDelta"` } -// Implement Record interface for RelationRecord. func (r *RelationRecord) GetCheckpointID() int64 { return r.CheckpointID } @@ -217,6 +231,10 @@ func (r *RelationRecord) GetDestinationTableName() string { return r.TableSchemaDelta.DstTableName } +func (r *RelationRecord) GetSourceTableName() string { + return r.TableSchemaDelta.SrcTableName +} + func (r *RelationRecord) GetItems() *RecordItems { return nil } diff --git a/flow/model/record_items.go b/flow/model/record_items.go index fe9da58af9..a258f78be5 100644 --- a/flow/model/record_items.go +++ b/flow/model/record_items.go @@ -236,8 +236,7 @@ func (r *RecordItems) ToJSONWithOptions(options *ToJSONOptions) (string, error) } func (r *RecordItems) ToJSON() (string, error) { - unnestCols := make([]string, 0) - return r.ToJSONWithOpts(NewToJSONOptions(unnestCols, true)) + return r.ToJSONWithOpts(NewToJSONOptions(nil, true)) } func (r *RecordItems) ToJSONWithOpts(opts *ToJSONOptions) (string, error) { diff --git a/flow/pua/flatbuffers.go b/flow/pua/flatbuffers.go new file mode 100644 index 0000000000..a6c8c7a23a --- /dev/null +++ b/flow/pua/flatbuffers.go @@ -0,0 +1,47 @@ +package pua + +import ( + "github.com/yuin/gopher-lua" +) + +/* +local m = {} + +m.Builder = require("flatbuffers.builder").New +m.N = require("flatbuffers.numTypes") +m.view = require("flatbuffers.view") +m.binaryArray = require("flatbuffers.binaryarray") + +return m +*/ + +func requireHelper(ls *lua.LState, m *lua.LTable, require lua.LValue, name string, path string) { + ls.Push(require) + ls.Push(lua.LString(path)) + ls.Call(1, 1) + ls.SetField(m, name, ls.Get(-1)) + ls.Pop(1) +} + +func FlatBuffers_Loader(ls *lua.LState) int { + ls.PreloadModule("flatbuffers.binaryarray", FlatBuffers_BinaryArray_Loader) + ls.PreloadModule("flatbuffers.builder", FlatBuffers_Builder_Loader) + ls.PreloadModule("flatbuffers.numTypes", FlatBuffers_N_Loader) + ls.PreloadModule("flatbuffers.view", FlatBuffers_View_Loader) + + m := ls.NewTable() + require := ls.GetGlobal("require") + ls.Push(require) + ls.Push(lua.LString("flatbuffers.builder")) + ls.Call(1, 1) + builder := ls.GetTable(ls.Get(-1), lua.LString("New")) + ls.SetField(m, "builder", builder) + ls.Pop(1) + + requireHelper(ls, m, require, "N", "flatbuffers.numTypes") + requireHelper(ls, m, require, "view", "flatbuffers.view") + requireHelper(ls, m, require, "binaryArray", "flatbuffers.binaryarray") + + ls.Push(m) + return 1 +} diff --git a/flow/pua/flatbuffers_binaryarray.go b/flow/pua/flatbuffers_binaryarray.go new file mode 100644 index 0000000000..e296c871bf --- /dev/null +++ b/flow/pua/flatbuffers_binaryarray.go @@ -0,0 +1,129 @@ +package pua + +import ( + "github.com/yuin/gopher-lua" +) + +type BinaryArray struct { + data []byte +} + +var LuaBinaryArray = LuaUserDataType[BinaryArray]{Name: "flatbuffers_binaryarray"} + +func FlatBuffers_BinaryArray_Loader(ls *lua.LState) int { + m := ls.NewTable() + ls.SetField(m, "New", ls.NewFunction(BinaryArrayNew)) + + mt := LuaBinaryArray.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(BinaryArrayIndex)) + ls.SetField(mt, "__len", ls.NewFunction(BinaryArrayLen)) + ls.SetField(mt, "Slice", ls.NewFunction(BinaryArraySlice)) + ls.SetField(mt, "Grow", ls.NewFunction(BinaryArrayGrow)) + ls.SetField(mt, "Pad", ls.NewFunction(BinaryArrayPad)) + ls.SetField(mt, "Set", ls.NewFunction(BinaryArraySet)) + + ls.Push(m) + return 1 +} + +func BinaryArrayNew(ls *lua.LState) int { + lval := ls.Get(-1) + var ba BinaryArray + switch val := lval.(type) { + case lua.LString: + ba = BinaryArray{ + data: []byte(val), + } + case lua.LNumber: + ba = BinaryArray{ + data: make([]byte, int(val)), + } + default: + ls.RaiseError("Expect a integer size value or string to construct a binary array") + return 0 + } + ls.Push(LuaBinaryArray.New(ls, ba)) + return 1 +} + +func BinaryArrayLen(ls *lua.LState) int { + ba := LuaBinaryArray.StartMeta(ls) + ls.Push(lua.LNumber(len(ba.data))) + return 1 +} + +func BinaryArrayIndex(ls *lua.LState) int { + ba, key := LuaBinaryArray.StartIndex(ls) + switch key { + case "size": + ls.Push(lua.LNumber(len(ba.data))) + case "str": + ls.Push(lua.LString(ba.data)) + case "data": + ls.RaiseError("BinaryArray data property inaccessible") + return 0 + default: + ls.Push(ls.GetField(LuaBinaryArray.Metatable(ls), key)) + } + return 1 +} + +func BinaryArraySlice(ls *lua.LState) int { + var startPos, endPos int + ba := LuaBinaryArray.StartMeta(ls) + if luaStartPos, ok := ls.Get(2).(lua.LNumber); ok { + startPos = max(int(luaStartPos), 0) + } else { + startPos = 0 + } + if luaEndPos, ok := ls.Get(3).(lua.LNumber); ok { + endPos = min(int(luaEndPos), len(ba.data)) + } else { + endPos = len(ba.data) + } + ls.Push(lua.LString(ba.data[startPos:endPos])) + return 1 +} + +func (ba *BinaryArray) Grow(newsize int) { + newdata := make([]byte, newsize) + copy(newdata[newsize-len(ba.data):], ba.data) + ba.data = newdata +} + +func BinaryArrayGrow(ls *lua.LState) int { + baud, ba := LuaBinaryArray.Check(ls, 1) + newsize := int(ls.CheckNumber(2)) + if newsize > len(ba.data) { + ba.Grow(newsize) + baud.Value = ba + } + return 0 +} + +func (ba *BinaryArray) Pad(n int, start int) { + for i := range n { + ba.data[start+i] = 0 + } +} + +func BinaryArrayPad(ls *lua.LState) int { + ba := LuaBinaryArray.StartMeta(ls) + n := int(ls.CheckNumber(2)) + startPos := int(ls.CheckNumber(3)) + ba.Pad(n, startPos) + return 0 +} + +func BinaryArraySet(ls *lua.LState) int { + ba := LuaBinaryArray.StartMeta(ls) + idx := int(ls.CheckNumber(3)) + value := ls.Get(2) + if num, ok := value.(lua.LNumber); ok { + ba.data[idx] = byte(num) + } + if str, ok := value.(lua.LString); ok { + ba.data[idx] = str[0] + } + return 0 +} diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers_builder.go new file mode 100644 index 0000000000..6abf8f55fc --- /dev/null +++ b/flow/pua/flatbuffers_builder.go @@ -0,0 +1,591 @@ +package pua + +import ( + "github.com/yuin/gopher-lua" +) + +const VtableMetadataFields int = 2 + +type Builder struct { + ba BinaryArray + vtables []int + currentVT []int + head int + objectEnd int + finished bool + nested bool + minalign uint8 +} + +func (b *Builder) EndVector(vectorSize int) int { + if !b.nested { + panic("EndVector called outside nested context") + } + b.PlaceU64(uint64(vectorSize), uint32n) + return b.Offset() +} + +func (b *Builder) Offset() int { + return len(b.ba.data) - b.head +} + +func (b *Builder) Pad(pad int) { + if pad > 0 { + b.head -= pad + b.ba.Pad(pad, b.head) + } +} + +func (b *Builder) Place(ls *lua.LState, x lua.LValue, n N) { + b.head -= int(n.width) + n.Pack(ls, b.ba.data[b.head:], x) +} + +func (b *Builder) PlaceU64(u64 uint64, n N) { + b.head -= int(n.width) + n.PackU64(b.ba.data[b.head:], u64) +} + +func (b *Builder) Prep(width uint8, additional int) { + if width > b.minalign { + b.minalign = width + } + k := len(b.ba.data) - b.head + additional + alignsize := (^k + 1) & int(width-1) + desiredSize := alignsize + int(width) + additional + + for b.head < desiredSize { + oldBufSize := len(b.ba.data) + newBufSize := oldBufSize + 1 + for newBufSize < desiredSize { + newBufSize *= 2 + } + b.ba.Grow(newBufSize) + b.head += len(b.ba.data) - oldBufSize + } + + b.Pad(alignsize) +} + +func (b *Builder) Prepend(ls *lua.LState, n N, x lua.LValue) { + b.Prep(n.width, 0) + b.Place(ls, x, n) +} + +func (b *Builder) PrependU64(n N, x uint64) { + b.Prep(n.width, 0) + b.PlaceU64(x, n) +} + +func (b *Builder) PrependSlot(ls *lua.LState, n N, slotnum int, x lua.LValue, d lua.LValue) { + // TODO implement __eq for U64/I64 + if !ls.Equal(x, d) { + b.Prepend(ls, n, x) + b.Slot(slotnum) + } +} + +func (b *Builder) PrependOffsetTRelative(ls *lua.LState, off int, n N) { + b.Prep(4, 0) + boff := b.Offset() + if off > boff { + ls.RaiseError("Offset arithmetic error") + } else { + b.PlaceU64(uint64(boff-off+4), n) + } +} + +func (b *Builder) PrependSOffsetTRelative(ls *lua.LState, off int) { + b.PrependOffsetTRelative(ls, off, int32n) +} + +func (b *Builder) PrependUOffsetTRelative(ls *lua.LState, off int) { + b.PrependOffsetTRelative(ls, off, uint32n) +} + +func (b *Builder) PrependVOffsetT(off uint16) { + b.Prep(2, 0) + b.PlaceU64(uint64(off), uint16n) +} + +func (b *Builder) Slot(slotnum int) { + if !b.nested { + panic("Slot called outside nested context") + } + for slotnum < len(b.currentVT) { + b.currentVT = append(b.currentVT, 0) + } + b.currentVT[slotnum] = b.Offset() +} + +func vtableEqual(a []int, objectStart int, b []byte) bool { + if len(a)*2 != len(b) { + return false + } + + for i, ai := range a { + x := int16n.UnpackU64(b[i*2:]) + if (x != 0 || ai != 0) && int(x) != objectStart-ai { + return false + } + } + return true +} + +func (b *Builder) WriteVtable(ls *lua.LState) int { + b.PrependSOffsetTRelative(ls, 0) + objectOffset := b.Offset() + + for len(b.ba.data) > 0 && b.ba.data[len(b.ba.data)-1] == 0 { + b.ba.data = b.ba.data[:len(b.ba.data)-1] + } + + var existingVtable int + for i := len(b.ba.data) - 1; i >= 0; i-- { + vt2Offset := b.vtables[i] + vt2Start := len(b.ba.data) - vt2Offset + vt2Len := uint16n.UnpackU64(b.ba.data[vt2Start:]) + + metadata := VtableMetadataFields * 2 + vt2End := vt2Start + int(vt2Len) + vt2 := b.ba.data[vt2Start+metadata : vt2End] + + if vtableEqual(b.currentVT, objectOffset, vt2) { + existingVtable = vt2Offset + break + } + } + + if existingVtable == 0 { + for i := len(b.currentVT) - 1; i >= 0; i-- { + var off uint16 + if b.currentVT[i] != 0 { + off = uint16(objectOffset - b.currentVT[i]) + } + b.PrependVOffsetT(off) + } + + objectSize := uint16(objectOffset - b.objectEnd) + b.PrependVOffsetT(objectSize) + + vBytes := uint16(len(b.currentVT)+VtableMetadataFields) * 2 + b.PrependVOffsetT(vBytes) + + objectStart := len(b.ba.data) - objectOffset + newOffset := b.Offset() + int32n.PackU64(b.ba.data[newOffset-objectOffset:], uint64(objectStart)) + + b.vtables = append(b.vtables, newOffset) + } else { + b.head = len(b.ba.data) - objectOffset + int16n.PackU64(b.ba.data[b.head:], uint64(existingVtable-objectOffset)) + } + + if len(b.currentVT) != 0 { + b.currentVT = b.currentVT[:0] + } + return objectOffset +} + +var LuaBuilder = LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} + +func FlatBuffers_Builder_Loader(ls *lua.LState) int { + m := ls.NewTable() + ls.SetField(m, "New", ls.NewFunction(BuilderNew)) + + mt := LuaBinaryArray.NewMetatable(ls) + index := ls.SetFuncs(ls.NewTable(), map[string]lua.LGFunction{ + "Clear": BuilderClear, + "Output": BuilderOutput, + "StartObject": BuilderStartObject, + "WriteVtable": BuilderWriteVtable, + "EndObject": BuilderEndObject, + "Head": BuilderHead, + "Offset": BuilderOffset, + "Pad": BuilderPad, + "Prep": BuilderPrep, + "StartVector": BuilderStartVector, + "EndVector": BuilderEndVector, + "CreateString": BuilderCreateString, + "CreateByteVector": BuilderCreateString, + "Slot": BuilderSlot, + "Finish": BuilderFinish, + "FinishSizePrefixed": BuilderFinishSizePrefixed, + "Place": BuilderPlace, + + "PrependSlot": BuilderPrependSlot, + "PrependBoolSlot": BuilderPrependBoolSlot, + "PrependByteSlot": BuilderPrependUint8Slot, + "PrependUint8Slot": BuilderPrependUint8Slot, + "PrependUint16Slot": BuilderPrependUint16Slot, + "PrependUint32Slot": BuilderPrependUint32Slot, + "PrependUint64Slot": BuilderPrependUint64Slot, + "PrependInt8Slot": BuilderPrependInt8Slot, + "PrependInt16Slot": BuilderPrependInt16Slot, + "PrependInt32Slot": BuilderPrependInt32Slot, + "PrependInt64Slot": BuilderPrependInt64Slot, + "PrependFloat32Slot": BuilderPrependFloat32Slot, + "PrependFloat64Slot": BuilderPrependFloat64Slot, + "PrependStructSlot": BuilderPrependStructSlot, + "PrependUOffsetTRelativeSlot": BuilderPrependUOffsetTRelativeSlot, + + "Prepend": BuilderPrepend, + "PrependBool": BuilderPrependBool, + "PrependByte": BuilderPrependUint8, + "PrependUint8": BuilderPrependUint8, + "PrependUint16": BuilderPrependUint16, + "PrependUint32": BuilderPrependUint32, + "PrependUint64": BuilderPrependUint64, + "PrependInt8": BuilderPrependInt8, + "PrependInt16": BuilderPrependInt16, + "PrependInt32": BuilderPrependInt32, + "PrependInt64": BuilderPrependInt64, + "PrependFloat32": BuilderPrependFloat32, + "PrependFloat64": BuilderPrependFloat64, + "PrependVOffsetT": BuilderPrependUint16, + "PrependSOffsetTRelative": BuilderPrependSOffsetTRelative, + "PrependUOffsetTRelative": BuilderPrependUOffsetTRelative, + }) + ls.SetField(mt, "__index", index) + + ls.Push(m) + return 1 +} + +func BuilderNew(ls *lua.LState) int { + initialSize := int(ls.CheckNumber(1)) + + ls.Push(LuaBuilder.New(ls, &Builder{ + ba: BinaryArray{data: make([]byte, initialSize)}, + vtables: nil, + currentVT: nil, + head: initialSize, + objectEnd: 0, + finished: false, + nested: false, + minalign: 1, + })) + return 1 +} + +func BuilderClear(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + b.finished = false + b.nested = false + b.minalign = 1 + if len(b.vtables) != 0 { + b.vtables = b.vtables[:0] + } + if len(b.currentVT) != 0 { + b.currentVT = b.currentVT[:0] + } + b.objectEnd = 0 + b.head = len(b.ba.data) + return 0 +} + +func BuilderOutput(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + if lua.LVIsFalse(ls.Get(2)) { + ls.Push(lua.LString(b.ba.data[b.head:])) + } else { + ls.Push(lua.LString(b.ba.data)) + } + return 1 +} + +func BuilderStartObject(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + if b.nested { + ls.RaiseError("StartObject called inside nested context") + return 0 + } + + numFields := int(ls.CheckNumber(2)) + b.currentVT = make([]int, numFields) + b.objectEnd = b.Offset() + b.nested = true + return 0 +} + +func BuilderWriteVtable(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + b.WriteVtable(ls) + return 0 +} + +func BuilderEndObject(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + if !b.nested { + panic("EndObject called outside nested context") + } + ls.Push(lua.LNumber(b.WriteVtable(ls))) + return 1 +} + +func BuilderHead(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + ls.Push(lua.LNumber(b.head)) + return 1 +} + +func BuilderOffset(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + ls.Push(lua.LNumber(len(b.ba.data) - b.head)) + return 1 +} + +func BuilderPad(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + pad := ls.CheckNumber(2) + b.Pad(int(pad)) + return 0 +} + +func BuilderPrep(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + size := ls.CheckNumber(2) + additional := ls.CheckNumber(3) + b.Prep(uint8(size), int(additional)) + return 0 +} + +func BuilderPrependSOffsetTRelative(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + b.PrependOffsetTRelative(ls, int(ls.CheckNumber(2)), int32n) + return 0 +} + +func BuilderPrependUOffsetTRelative(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + b.PrependOffsetTRelative(ls, int(ls.CheckNumber(2)), uint32n) + return 0 +} + +func BuilderStartVector(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + if b.nested { + ls.RaiseError("StartVector called in nested context") + } + b.nested = true + elemSize := int(ls.CheckNumber(2)) + numElements := int(ls.CheckNumber(3)) + alignment := uint8(ls.CheckNumber(4)) + elementSize := elemSize * numElements + b.Prep(4, elementSize) + b.Prep(alignment, elementSize) + ls.Push(lua.LNumber(b.Offset())) + return 1 +} + +func BuilderEndVector(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + if !b.nested { + ls.RaiseError("EndVector called outside nested context") + } + b.nested = false + b.PlaceU64(uint64(ls.CheckNumber(2)), uint32n) + ls.Push(lua.LNumber(b.Offset())) + return 1 +} + +func BuilderCreateString(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + s := ls.CheckString(2) + if b.nested { + ls.RaiseError("CreateString called in nested context") + return 0 + } + b.nested = true + + lens := len(s) + b.Prep(4, lens+1) + b.PlaceU64(0, uint8n) + b.head -= lens + copy(b.ba.data[b.head:], s) + + return b.EndVector(lens) +} + +func BuilderSlot(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + slotnum := int(ls.CheckNumber(2)) + b.Slot(slotnum) + return 0 +} + +func FinishHelper(ls *lua.LState, sizePrefix bool) int { + b := LuaBuilder.StartMeta(ls) + rootTable := int(ls.CheckNumber(2)) + var additional int + if sizePrefix { + additional = 8 + } else { + additional = 4 + } + b.Prep(b.minalign, additional) + b.PrependUOffsetTRelative(ls, rootTable) + if sizePrefix { + size := len(b.ba.data) - b.head + b.PrependU64(int32n, uint64(size)) + } + b.finished = true + ls.Push(lua.LNumber(b.head)) + return 1 +} + +func BuilderFinish(ls *lua.LState) int { + return FinishHelper(ls, false) +} + +func BuilderFinishSizePrefixed(ls *lua.LState) int { + return FinishHelper(ls, true) +} + +func BuilderPlace(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + _, n := LuaN.Check(ls, 3) + b.Place(ls, ls.Get(2), n) + return 0 +} + +func BuilderPrependSlot(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + _, n := LuaN.Check(ls, 2) + slotnum := int(ls.CheckNumber(3)) + b.PrependSlot(ls, n, slotnum, ls.Get(4), ls.Get(5)) + return 0 +} + +func PrependSlotHelper(ls *lua.LState, n N) int { + b := LuaBuilder.StartMeta(ls) + slotnum := int(ls.CheckNumber(2)) + b.PrependSlot(ls, n, slotnum, ls.Get(3), ls.Get(4)) + return 0 +} + +func BuilderPrependBoolSlot(ls *lua.LState) int { + return PrependSlotHelper(ls, booln) +} + +func BuilderPrependUint8Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, uint8n) +} + +func BuilderPrependUint16Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, uint16n) +} + +func BuilderPrependUint32Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, uint32n) +} + +func BuilderPrependUint64Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, uint64n) +} + +func BuilderPrependInt8Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, int8n) +} + +func BuilderPrependInt16Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, int16n) +} + +func BuilderPrependInt32Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, int32n) +} + +func BuilderPrependInt64Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, int64n) +} + +func BuilderPrependFloat32Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, float32n) +} + +func BuilderPrependFloat64Slot(ls *lua.LState) int { + return PrependSlotHelper(ls, float64n) +} + +func BuilderPrependStructSlot(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + x := int(ls.CheckNumber(3)) + d := int(ls.CheckNumber(4)) + if x != d { + if x != b.Offset() { + ls.RaiseError("Tried to write a Struct at an Offset that is different from the current Offset of the Builder.") + } else { + b.Slot(int(ls.CheckNumber(2))) + } + } + return 0 +} + +func BuilderPrependUOffsetTRelativeSlot(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + x := int(ls.CheckNumber(3)) + d := int(ls.CheckNumber(4)) + if x != d { + b.PrependOffsetTRelative(ls, x, uint32n) + b.Slot(int(ls.CheckNumber(2))) + } + return 0 +} + +func BuilderPrepend(ls *lua.LState) int { + b := LuaBuilder.StartMeta(ls) + _, n := LuaN.Check(ls, 2) + b.Prepend(ls, n, ls.Get(3)) + return 0 +} + +func PrependHelper(ls *lua.LState, n N) int { + b := LuaBuilder.StartMeta(ls) + b.Prepend(ls, n, ls.Get(2)) + return 0 +} + +func BuilderPrependBool(ls *lua.LState) int { + return PrependHelper(ls, booln) +} + +func BuilderPrependUint8(ls *lua.LState) int { + return PrependHelper(ls, uint8n) +} + +func BuilderPrependUint16(ls *lua.LState) int { + return PrependHelper(ls, uint16n) +} + +func BuilderPrependUint32(ls *lua.LState) int { + return PrependHelper(ls, uint32n) +} + +func BuilderPrependUint64(ls *lua.LState) int { + return PrependHelper(ls, uint64n) +} + +func BuilderPrependInt8(ls *lua.LState) int { + return PrependHelper(ls, int8n) +} + +func BuilderPrependInt16(ls *lua.LState) int { + return PrependHelper(ls, int16n) +} + +func BuilderPrependInt32(ls *lua.LState) int { + return PrependHelper(ls, int32n) +} + +func BuilderPrependInt64(ls *lua.LState) int { + return PrependHelper(ls, int64n) +} + +func BuilderPrependFloat32(ls *lua.LState) int { + return PrependHelper(ls, float32n) +} + +func BuilderPrependFloat64(ls *lua.LState) int { + return PrependHelper(ls, float64n) +} diff --git a/flow/pua/flatbuffers_numtypes.go b/flow/pua/flatbuffers_numtypes.go new file mode 100644 index 0000000000..b174bee873 --- /dev/null +++ b/flow/pua/flatbuffers_numtypes.go @@ -0,0 +1,235 @@ +package pua + +import ( + "encoding/binary" + "math" + "strconv" + + "github.com/yuin/gopher-lua" +) + +// Minimal API implemented for generated code + +type Ntype = uint8 + +const ( + tyint Ntype = 0 + tyfloat Ntype = 1 + tybool Ntype = 2 +) + +type N struct { + width uint8 + signed bool + ntype Ntype +} + +var ( + uint8n = N{width: 1, signed: false, ntype: tyint} + uint16n = N{width: 2, signed: false, ntype: tyint} + uint32n = N{width: 4, signed: false, ntype: tyint} + uint64n = N{width: 8, signed: false, ntype: tyint} + int8n = N{width: 1, signed: true, ntype: tyint} + int16n = N{width: 2, signed: true, ntype: tyint} + int32n = N{width: 4, signed: true, ntype: tyint} + int64n = N{width: 8, signed: true, ntype: tyint} + float32n = N{width: 4, signed: true, ntype: tyfloat} + float64n = N{width: 8, signed: true, ntype: tyfloat} + booln = N{width: 1, signed: false, ntype: tybool} +) + +func (n *N) PackU64(buf []byte, val uint64) { + switch n.width { + case 1: + buf[0] = uint8(val) + case 2: + binary.LittleEndian.PutUint16(buf, uint16(val)) + case 4: + binary.LittleEndian.PutUint32(buf, uint32(val)) + case 8: + binary.LittleEndian.PutUint64(buf, val) + default: + panic("Invalid PackU64 width") + } +} + +func (n *N) Pack(ls *lua.LState, buf []byte, val lua.LValue) { + switch n.ntype { + case tyint: + switch lv := val.(type) { + case *lua.LUserData: + switch v := lv.Value.(type) { + case NI64: + n.PackU64(buf, uint64(v.val)) + case NU64: + n.PackU64(buf, v.val) + default: + n.PackU64(buf, 0) + } + case lua.LNumber: + n.PackU64(buf, uint64(lv)) + case lua.LString: + sv := string(lv) + u64, err := strconv.ParseUint(sv, 10, int(n.width)*8) + if err != nil { + i64, err := strconv.ParseInt(sv, 10, int(n.width)*8) + if err != nil { + n.PackU64(buf, 0) + } else { + n.PackU64(buf, uint64(i64)) + } + } else { + n.PackU64(buf, u64) + } + default: + n.PackU64(buf, 0) + } + case tyfloat: + switch lv := val.(type) { + case *lua.LUserData: + switch v := lv.Value.(type) { + case NI64: + n.PackU64(buf, math.Float64bits(float64(v.val))) + case NU64: + n.PackU64(buf, math.Float64bits(float64(v.val))) + default: + n.PackU64(buf, 0) + } + case lua.LNumber: + n.PackU64(buf, math.Float64bits(float64(lv))) + case lua.LString: + f64, err := strconv.ParseFloat(string(lv), int(n.width)*8) + if err != nil { + n.PackU64(buf, math.Float64bits(f64)) + } else { + n.PackU64(buf, 0) + } + default: + n.PackU64(buf, 0) + } + case tybool: + if lua.LVIsFalse(val) { + buf[0] = 0 + } else { + buf[1] = 1 + } + } +} + +func (n *N) UnpackU64(buf []byte) uint64 { + switch n.width { + case 1: + return uint64(buf[0]) + case 2: + return uint64(binary.LittleEndian.Uint16(buf)) + case 4: + return uint64(binary.LittleEndian.Uint32(buf)) + case 8: + return binary.LittleEndian.Uint64(buf) + } + panic("invalid bitwidth") +} + +func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { + switch n.ntype { + case tyint: + if !n.signed && n.width < 8 { + return lua.LNumber(n.UnpackU64(buf)) + } + switch n.width { + case 1: + return lua.LNumber(int8(buf[0])) + case 2: + return lua.LNumber(int16(binary.LittleEndian.Uint16(buf))) + case 4: + return lua.LNumber(int32(binary.LittleEndian.Uint32(buf))) + case 8: + u64 := binary.LittleEndian.Uint64(buf) + if n.signed { + return LuaNI64.New(ls, NI64{int64(u64)}) + } else { + return LuaNU64.New(ls, NU64{u64}) + } + } + case tyfloat: + if n.width == 4 { + u32 := binary.LittleEndian.Uint32(buf) + return lua.LNumber(math.Float32frombits(u32)) + } else { + u64 := binary.LittleEndian.Uint64(buf) + return lua.LNumber(math.Float64frombits(u64)) + } + case tybool: + return lua.LBool(buf[0] != 0) + } + panic("invalid numeric metatype") +} + +type ( + NI64 struct{ val int64 } + NU64 struct{ val uint64 } +) + +var ( + LuaN = LuaUserDataType[N]{Name: "flatbuffers_n"} + LuaNI64 = LuaUserDataType[NI64]{Name: "flatbuffers_i64"} + LuaNU64 = LuaUserDataType[NU64]{Name: "flatbuffers_u64"} +) + +func FlatBuffers_N_Loader(ls *lua.LState) int { + m := ls.NewTable() + + mt := LuaView.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(NIndex)) + + uint16ud := LuaN.New(ls, uint16n) + uint32ud := LuaN.New(ls, uint32n) + int32ud := LuaN.New(ls, int32n) + + ls.SetField(m, "Uint8", LuaN.New(ls, uint8n)) + ls.SetField(m, "Uint16", uint16ud) + ls.SetField(m, "Uint32", uint32ud) + ls.SetField(m, "Uint64", LuaN.New(ls, uint64n)) + ls.SetField(m, "Int8", LuaN.New(ls, int8n)) + ls.SetField(m, "Int16", LuaN.New(ls, int16n)) + ls.SetField(m, "Int32", int32ud) + ls.SetField(m, "Int64", LuaN.New(ls, int64n)) + ls.SetField(m, "Float32", LuaN.New(ls, float32n)) + ls.SetField(m, "Float64", LuaN.New(ls, float64n)) + ls.SetField(m, "Bool", LuaN.New(ls, booln)) + + ls.SetField(m, "UOffsetT", uint32ud) + ls.SetField(m, "VOffsetT", uint16ud) + ls.SetField(m, "SOffsetT", int32ud) + + ls.Push(m) + return 1 +} + +func NIndex(ls *lua.LState) int { + n, key := LuaN.StartIndex(ls) + if key == "Unpack" { + var buf []byte + switch v := ls.Get(1).(type) { + case lua.LString: + buf = []byte(v) + case *lua.LUserData: + ba, ok := v.Value.(BinaryArray) + if ok { + buf = ba.data + } else { + ls.RaiseError("Invalid buf userdata passed to unpack") + return 0 + } + default: + ls.RaiseError("Invalid buf passed to unpack") + return 0 + } + pos := max(CheckOffset(ls, 2), 1) + ls.Push(n.Unpack(ls, buf[pos-1:])) + return 1 + } else { + ls.RaiseError("Unsupported field on N: " + key) + return 0 + } +} diff --git a/flow/pua/flatbuffers_view.go b/flow/pua/flatbuffers_view.go new file mode 100644 index 0000000000..985fa84f0d --- /dev/null +++ b/flow/pua/flatbuffers_view.go @@ -0,0 +1,224 @@ +package pua + +import ( + "github.com/yuin/gopher-lua" +) + +type View struct { + ba BinaryArray + pos int // 0-based offset + vtable int // 0-based offset + vtableEnd uint16 + hasv bool +} + +/* +func (view *View) Get(ls *lua.LState, n N, offset int) lua.LValue { + return n.Unpack(ls, view.ba.data[offset-1:]) +} +*/ + +var LuaView = LuaUserDataType[*View]{Name: "flatbuffers_view"} + +func CheckOffset(ls *lua.LState, idx int) int { + num := ls.CheckNumber(idx) + if num < 0 || num > 42949672951 { + ls.RaiseError("Offset is not valid") + } + return int(num) +} + +func FlatBuffers_View_Loader(ls *lua.LState) int { + m := ls.NewTable() + ls.SetField(m, "New", ls.NewFunction(ViewNew)) + + mt := LuaView.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(ViewIndex)) + + ls.Push(m) + return 1 +} + +func ViewNew(ls *lua.LState) int { + buf := ls.Get(1) + var ba BinaryArray + switch val := buf.(type) { + case lua.LString: + ba = BinaryArray{data: []byte(val)} + case *lua.LUserData: + var ok bool + ba, ok = val.Value.(BinaryArray) + if !ok { + ls.RaiseError("invalid buf userdata passed to view.New") + return 0 + } + default: + ls.RaiseError("invalid buf passed to view.New") + return 0 + } + ls.Push(LuaView.New(ls, &View{ + ba: ba, + pos: CheckOffset(ls, 2), + })) + return 1 +} + +func ViewIndex(ls *lua.LState) int { + view, key := LuaView.StartIndex(ls) + switch key { + case "bytes": + ls.Push(LuaBinaryArray.New(ls, view.ba)) + case "pos": + ls.Push(lua.LNumber(view.pos)) + case "Offset": + ls.Push(ls.NewFunction(ViewOffset)) + case "Indirect": + ls.Push(ls.NewFunction(ViewIndirect)) + case "String": + ls.Push(ls.NewFunction(ViewString)) + case "VectorLen": + ls.Push(ls.NewFunction(ViewVectorLen)) + case "Vector": + ls.Push(ls.NewFunction(ViewVector)) + case "VectorAsString": + ls.Push(ls.NewFunction(ViewVectorAsString)) + case "Union": + ls.Push(ls.NewFunction(ViewUnion)) + case "Get": + ls.Push(ls.NewFunction(ViewGet)) + case "GetSlot": + ls.Push(ls.NewFunction(ViewGetSlot)) + case "GetVOffsetTSlot": + ls.Push(ls.NewFunction(ViewGetVOffsetTSlot)) + } + return 1 +} + +func (view *View) Offset(vtoff uint16) uint16 { + if !view.hasv { + view.vtable = view.pos - int(int32(int32n.UnpackU64(view.ba.data[view.pos:]))) + view.vtableEnd = uint16(uint16n.UnpackU64(view.ba.data[view.vtable:])) + view.hasv = true + } + if vtoff < view.vtableEnd { + return uint16(uint16n.UnpackU64(view.ba.data[view.vtable+int(vtoff):])) + } else { + return 0 + } +} + +func (view *View) Vector(off int) int { + off += view.pos + return off + int(uint32n.UnpackU64(view.ba.data[off:])) + 4 +} + +func (view *View) VectorLen(off int) uint32 { + off += int(uint32n.UnpackU64(view.ba.data[view.pos+off:])) + return uint32(uint32n.UnpackU64(view.ba.data[off:])) +} + +func ViewOffset(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + vtoff := uint16(CheckOffset(ls, 2)) + ls.Push(lua.LNumber(view.Offset(vtoff))) + return 1 +} + +func ViewIndirect(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + off := CheckOffset(ls, 2) + ls.Push(lua.LNumber(off + int(uint32n.UnpackU64(view.ba.data[off:])))) + return 1 +} + +func ViewString(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + off := CheckOffset(ls, 2) + off += int(uint32n.UnpackU64(view.ba.data[off:])) + start := off + 4 + length := int(uint32n.UnpackU64(view.ba.data[off:])) + ls.Push(lua.LString(view.ba.data[start : start+length])) + return 1 +} + +func ViewVectorLen(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + off := CheckOffset(ls, 2) + ls.Push(lua.LNumber(view.VectorLen(off))) + return 1 +} + +func ViewVector(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + off := CheckOffset(ls, 2) + ls.Push(lua.LNumber(view.Vector(off))) + return 1 +} + +func ViewVectorAsString(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + off := uint16(CheckOffset(ls, 2)) + o := view.Offset(off) + if o == 0 { + ls.Push(lua.LNil) + return 1 + } + var start, stop int + lstart, ok := ls.Get(3).(lua.LNumber) + if ok { + start = int(lstart) + } + lstop, ok := ls.Get(4).(lua.LNumber) + if ok { + stop = int(lstop) + } else { + stop = int(view.VectorLen(int(o))) + } + a := view.Vector(int(o)) + start + ls.Push(lua.LString(view.ba.data[a : a+stop-start])) + return 1 +} + +func ViewUnion(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + t2ud, t2 := LuaView.Check(ls, 2) + off := CheckOffset(ls, 3) + off += view.pos + t2.pos = off + int(uint32n.UnpackU64(view.ba.data[off:])) + t2.ba = view.ba + t2ud.Value = t2 + return 0 +} + +func ViewGet(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + _, n := LuaN.Check(ls, 2) + off := CheckOffset(ls, 3) + ls.Push(n.Unpack(ls, view.ba.data[off-1:])) + return 1 +} + +func ViewGetSlot(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + slot := uint16(CheckOffset(ls, 2)) + off := view.Offset(slot) + if off == 0 { + ls.Push(ls.Get(3)) + return 1 + } + _, validatorFlags := LuaN.Check(ls, 4) + ls.Push(validatorFlags.Unpack(ls, view.ba.data[view.pos+int(off):])) + return 1 +} + +func ViewGetVOffsetTSlot(ls *lua.LState) int { + view := LuaView.StartMeta(ls) + slot := uint16(CheckOffset(ls, 2)) + off := view.Offset(slot) + if off == 0 { + ls.Push(ls.Get(3)) + } else { + ls.Push(lua.LNumber(off)) + } + return 1 +} diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go new file mode 100644 index 0000000000..a42eb8dd48 --- /dev/null +++ b/flow/pua/peerdb.go @@ -0,0 +1,202 @@ +package pua + +import ( + "bytes" + "fmt" + "reflect" + "strings" + "time" + + "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" +) + +var ( + LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} + LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} + LuaQValue = LuaUserDataType[qvalue.QValue]{Name: "peerdb_value"} +) + +func RegisterTypes(ls *lua.LState) { + // gopher-lua provides 2 loaders {preload, file} + // overwrite file loader with one retrieving scripts from database + loaders := ls.GetField(ls.Get(lua.RegistryIndex), "_LOADERS").(*lua.LTable) + ls.RawSetInt(loaders, 2, ls.NewFunction(LoadPeerdbScript)) + + mt := LuaRecord.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(LuaRecordIndex)) + + mt = LuaRow.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(LuaRowIndex)) + ls.SetField(mt, "__len", ls.NewFunction(LuaRowLen)) + + mt = LuaQValue.NewMetatable(ls) + ls.SetField(mt, "__index", ls.NewFunction(LuaQValueIndex)) + ls.SetField(mt, "__len", ls.NewFunction(LuaQValueLen)) + + peerdb := ls.NewTable() + ls.SetField(peerdb, "RowToJSON", ls.NewFunction(LuaRowToJSON)) + ls.SetField(peerdb, "RowColumns", ls.NewFunction(LuaRowColumns)) + ls.SetField(peerdb, "UnixNow", ls.NewFunction(LuaUnixNow)) + ls.SetGlobal("peerdb", peerdb) +} + +func LoadPeerdbScript(ls *lua.LState) int { + ctx := ls.Context() + name := ls.CheckString(1) + pool, err := utils.GetCatalogConnectionPoolFromEnv(ctx) + if err != nil { + ls.RaiseError("Connection failed loading %s: %s", name, err.Error()) + return 0 + } + var source []byte + err = pool.QueryRow(ctx, "select source from scripts where lang = 'lua' and name = $1", name).Scan(&source) + if err == nil { + fn, err := ls.Load(bytes.NewReader(source), name) + if err != nil { + ls.RaiseError(err.Error()) + } + ls.Push(fn) + } + return 1 +} + +func LuaRowIndex(ls *lua.LState) int { + row, key := LuaRow.StartIndex(ls) + + qv, err := row.GetValueByColName(key) + if err != nil { + ls.RaiseError(err.Error()) + return 0 + } + + ls.Push(LuaQValue.New(ls, qv)) + return 1 +} + +func LuaRowLen(ls *lua.LState) int { + _, row := LuaRow.Check(ls, 1) + ls.Push(lua.LNumber(len(row.Values))) + return 1 +} + +func LuaRowToJSON(ls *lua.LState) int { + _, row := LuaRow.Check(ls, 1) + json, err := row.ToJSON() + if err != nil { + ls.RaiseError("failed to serialize json: %s", err.Error()) + return 0 + } + ls.Push(lua.LString(json)) + return 1 +} + +func LuaRowColumns(ls *lua.LState) int { + _, row := LuaRow.Check(ls, 1) + tbl := ls.CreateTable(len(row.ColToValIdx), 0) + for col, idx := range row.ColToValIdx { + ls.RawSetInt(tbl, idx, lua.LString(col)) + } + ls.Push(tbl) + return 1 +} + +func LuaRecordIndex(ls *lua.LState) int { + record, key := LuaRecord.StartIndex(ls) + switch key { + case "kind": + var tyname string + switch record.(type) { + case *model.InsertRecord: + tyname = "insert" + case *model.UpdateRecord: + tyname = "update" + case *model.DeleteRecord: + tyname = "delete" + case *model.RelationRecord: + tyname = "relation" + } + ls.Push(lua.LString(tyname)) + case "row": + items := record.GetItems() + if items != nil { + ls.Push(LuaRow.New(ls, items)) + } else { + ls.Push(lua.LNil) + } + case "old": + var items *model.RecordItems + switch rec := record.(type) { + case *model.UpdateRecord: + items = rec.OldItems + case *model.DeleteRecord: + items = rec.Items + } + if items != nil { + ls.Push(LuaRow.New(ls, items)) + } else { + ls.Push(lua.LNil) + } + case "new": + var items *model.RecordItems + switch rec := record.(type) { + case *model.InsertRecord: + items = rec.Items + case *model.UpdateRecord: + items = rec.NewItems + } + if items != nil { + ls.Push(LuaRow.New(ls, items)) + } else { + ls.Push(lua.LNil) + } + case "checkpoint": + ls.Push(LuaNI64.New(ls, NI64{record.GetCheckpointID()})) + case "target": + ls.Push(lua.LString(record.GetDestinationTableName())) + case "source": + ls.Push(lua.LString(record.GetSourceTableName())) + default: + return 0 + } + return 1 +} + +func LuaQValueIndex(ls *lua.LState) int { + qv, key := LuaQValue.StartIndex(ls) + switch key { + case "kind": + ls.Push(lua.LString(qv.Kind)) + case "int64": + ls.Push(LuaNI64.New(ls, NI64{reflect.ValueOf(qv.Value).Int()})) + case "float64": + ls.Push(lua.LNumber(reflect.ValueOf(qv.Value).Float())) + case "string": + ls.Push(lua.LString(fmt.Sprint(qv.Value))) + default: + return 0 + } + return 1 +} + +func LuaQValueLen(ls *lua.LState) int { + qv := LuaQValue.StartMeta(ls) + str, ok := qv.Value.(string) + if ok { + ls.Push(lua.LNumber(len(str))) + return 1 + } + if strings.HasPrefix(string(qv.Kind), "array_") { + ls.Push(lua.LNumber(reflect.ValueOf(qv.Value).Len())) + return 1 + } + return 0 +} + +func LuaUnixNow(ls *lua.LState) int { + ls.Push(lua.LNumber(float64(time.Now().UnixMilli()) / 1000.0)) + return 1 +} diff --git a/flow/pua/userdata.go b/flow/pua/userdata.go new file mode 100644 index 0000000000..5b87881fa8 --- /dev/null +++ b/flow/pua/userdata.go @@ -0,0 +1,41 @@ +package pua + +import ( + "github.com/yuin/gopher-lua" +) + +type LuaUserDataType[T any] struct{ Name string } + +func (udt *LuaUserDataType[T]) New(ls *lua.LState, val T) *lua.LUserData { + return &lua.LUserData{ + Value: val, + Env: ls.Env, + Metatable: udt.Metatable(ls), + } +} + +func (udt *LuaUserDataType[T]) NewMetatable(ls *lua.LState) *lua.LTable { + return ls.NewTypeMetatable(udt.Name) +} + +func (udt *LuaUserDataType[T]) Metatable(ls *lua.LState) lua.LValue { + return ls.GetTypeMetatable(udt.Name) +} + +func (udt *LuaUserDataType[T]) Check(ls *lua.LState, idx int) (*lua.LUserData, T) { + ud := ls.CheckUserData(idx) + val, ok := ud.Value.(T) + if !ok { + ls.RaiseError("Invalid " + udt.Name) + } + return ud, val +} + +func (udt *LuaUserDataType[T]) StartMeta(ls *lua.LState) T { + _, val := udt.Check(ls, 1) + return val +} + +func (udt *LuaUserDataType[T]) StartIndex(ls *lua.LState) (T, string) { + return udt.StartMeta(ls), ls.CheckString(2) +} diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 81468018a8..b2e5263937 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -10,8 +10,8 @@ use anyhow::Context; use pt::{ flow_model::{FlowJob, FlowJobTableMapping, QRepFlowJob}, peerdb_peers::{ - peer::Config, BigqueryConfig, ClickhouseConfig, DbType, EventHubConfig, MongoConfig, Peer, - PostgresConfig, S3Config, SnowflakeConfig, SqlServerConfig, + peer::Config, BigqueryConfig, ClickhouseConfig, DbType, EventHubConfig, KafkaConfig, + MongoConfig, Peer, PostgresConfig, S3Config, SnowflakeConfig, SqlServerConfig, }, }; use qrep::process_options; @@ -813,5 +813,33 @@ fn parse_db_options( }; Config::ClickhouseConfig(clickhouse_config) } + DbType::Kafka => { + let kafka_config = KafkaConfig { + servers: opts + .get("servers") + .context("no servers specified")? + .to_string() + .split(',') + .map(String::from) + .collect::>(), + username: opts + .get("user") + .context("no username specified")? + .to_string(), + password: opts + .get("password") + .context("no password specified")? + .to_string(), + sasl: opts + .get("sasl_mechanism") + .context("no sasl mechanism specified")? + .to_string(), + disable_tls: opts + .get("disable_tls") + .and_then(|s| s.parse::().ok()) + .unwrap_or_default(), + }; + Config::KafkaConfig(kafka_config) + } })) } diff --git a/nexus/catalog/migrations/V21__scripts.sql b/nexus/catalog/migrations/V21__scripts.sql new file mode 100644 index 0000000000..74060b25d6 --- /dev/null +++ b/nexus/catalog/migrations/V21__scripts.sql @@ -0,0 +1,8 @@ +CREATE TYPE script_lang AS ENUM ('lua'); + +CREATE TABLE IF NOT EXISTS scripts ( + id SERIAL PRIMARY KEY, + lang script_lang NOT NULL, + name TEXT NOT NULL UNIQUE, + source BYTEA NOT NULL +); diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index ce3eb7e778..4f6cace5f6 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -100,6 +100,7 @@ impl Catalog { eventhub_group_config.encode_to_vec() } Config::ClickhouseConfig(clickhouse_config) => clickhouse_config.encode_to_vec(), + Config::KafkaConfig(kafka_config) => kafka_config.encode_to_vec(), } }; @@ -306,6 +307,11 @@ impl Catalog { pt::peerdb_peers::ClickhouseConfig::decode(options).with_context(err)?; Config::ClickhouseConfig(clickhouse_config) } + DbType::Kafka => { + let kafka_config = + pt::peerdb_peers::KafkaConfig::decode(options).with_context(err)?; + Config::KafkaConfig(kafka_config) + } }) } else { None diff --git a/nexus/pt/src/lib.rs b/nexus/pt/src/lib.rs index 41b0566b2a..33da1e6382 100644 --- a/nexus/pt/src/lib.rs +++ b/nexus/pt/src/lib.rs @@ -25,7 +25,7 @@ impl From for DbType { PeerType::S3 => DbType::S3, PeerType::SQLServer => DbType::Sqlserver, PeerType::EventHubGroup => DbType::EventhubGroup, - PeerType::Kafka => todo!("Add Kafka support"), + PeerType::Kafka => DbType::Kafka, } } } diff --git a/protos/peers.proto b/protos/peers.proto index 063effc4da..a4c9fbc79d 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -108,6 +108,14 @@ message SqlServerConfig { string database = 5; } +message KafkaConfig { + repeated string servers = 1; + string username = 2; + string password = 3; + string sasl = 4; + bool disable_tls = 5; +} + enum DBType { BIGQUERY = 0; SNOWFLAKE = 1; @@ -118,6 +126,7 @@ enum DBType { SQLSERVER = 6; EVENTHUB_GROUP = 7; CLICKHOUSE = 8; + KAFKA = 9; } message Peer { @@ -133,5 +142,6 @@ message Peer { SqlServerConfig sqlserver_config = 9; EventHubGroupConfig eventhub_group_config = 10; ClickhouseConfig clickhouse_config = 11; + KafkaConfig kafka_config = 12; } } diff --git a/ui/README.md b/ui/README.md index 1d3fd5e120..6947d3f03d 100644 --- a/ui/README.md +++ b/ui/README.md @@ -2,7 +2,7 @@ PeerDB Cloud Template ## Prerequisites -- [NodeJS](https://nodejs.org/en): `18.17.1` +- [NodeJS](https://nodejs.org/en): `LTS` ## Getting Started diff --git a/ui/app/api/peers/getTruePeer.ts b/ui/app/api/peers/getTruePeer.ts index 3cef249e48..93da206854 100644 --- a/ui/app/api/peers/getTruePeer.ts +++ b/ui/app/api/peers/getTruePeer.ts @@ -4,6 +4,7 @@ import { ClickhouseConfig, EventHubConfig, EventHubGroupConfig, + KafkaConfig, Peer, PostgresConfig, S3Config, @@ -19,13 +20,14 @@ export const getTruePeer = (peer: CatalogPeer) => { const options = peer.options; let config: | BigqueryConfig - | SnowflakeConfig - | PostgresConfig + | ClickhouseConfig | EventHubConfig - | S3Config - | SqlServerConfig | EventHubGroupConfig - | ClickhouseConfig; + | KafkaConfig + | PostgresConfig + | S3Config + | SnowflakeConfig + | SqlServerConfig; switch (peer.type) { case 0: config = BigqueryConfig.decode(options); @@ -59,6 +61,10 @@ export const getTruePeer = (peer: CatalogPeer) => { config = ClickhouseConfig.decode(options); newPeer.clickhouseConfig = config; break; + case 9: + config = KafkaConfig.decode(options); + newPeer.kafkaConfig = config; + break; default: return newPeer; } diff --git a/ui/app/api/peers/info/[peerName]/route.ts b/ui/app/api/peers/info/[peerName]/route.ts index 93b0c00647..1e90bbdde6 100644 --- a/ui/app/api/peers/info/[peerName]/route.ts +++ b/ui/app/api/peers/info/[peerName]/route.ts @@ -1,15 +1,14 @@ import prisma from '@/app/utils/prisma'; -import { NextRequest, NextResponse } from 'next/server'; -import { getTruePeer } from '../../getTruePeer'; +import {NextRequest, NextResponse} from 'next/server'; -export async function GET( - request: NextRequest, - context: { params: { peerName: string } } -) { +import {getTruePeer} from '../../getTruePeer'; + +export async function GET(request: NextRequest, + context: {params: {peerName: string}}) { const peerName = context.params.peerName; const peer = await prisma.peers.findFirst({ - where: { - name: peerName, + where : { + name : peerName, }, }); const peerConfig = getTruePeer(peer!); @@ -20,6 +19,7 @@ export async function GET( const sfConfig = peerConfig.snowflakeConfig; const ehConfig = peerConfig.eventhubConfig; const chConfig = peerConfig.clickhouseConfig; + const kaConfig = peerConfig.kafkaConfig; if (pgConfig) { pgConfig.password = '********'; pgConfig.transactionSnapshot = '********'; @@ -42,6 +42,9 @@ export async function GET( chConfig.password = '********'; chConfig.secretAccessKey = '********'; } + if (kaConfig) { + kaConfig.password = '********'; + } return NextResponse.json(peerConfig); } diff --git a/ui/app/peers/create/[peerType]/handlers.ts b/ui/app/peers/create/[peerType]/handlers.ts index 423b996aec..42f84a492a 100644 --- a/ui/app/peers/create/[peerType]/handlers.ts +++ b/ui/app/peers/create/[peerType]/handlers.ts @@ -8,6 +8,7 @@ import { Dispatch, SetStateAction } from 'react'; import { bqSchema, chSchema, + kaSchema, peerNameSchema, pgSchema, s3Schema, @@ -57,6 +58,10 @@ const validateFields = ( const s3Config = s3Schema.safeParse(config); if (!s3Config.success) validationErr = s3Config.error.issues[0].message; break; + case 'KAFKA': + const kaConfig = kaSchema.safeParse(config); + if (!kaConfig.success) validationErr = kaConfig.error.issues[0].message; + break; default: validationErr = 'Unsupported peer type ' + type; } diff --git a/ui/app/peers/create/[peerType]/helpers/ka.ts b/ui/app/peers/create/[peerType]/helpers/ka.ts new file mode 100644 index 0000000000..7aa58e5642 --- /dev/null +++ b/ui/app/peers/create/[peerType]/helpers/ka.ts @@ -0,0 +1,33 @@ +import { KafkaConfig } from '@/grpc_generated/peers'; +import {PeerSetting } from './common'; + +export const kaSetting: PeerSetting[] = [ + { + label: 'Servers', + stateHandler: (value, setter) => setter((curr) => ({...cur, servers: value.split(',') })), + tips: 'Brokers', + helpfulLink: 'https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#SeedBrokers', + }, + { + label: 'Username', + stateHandler: (value, setter) => setter((curr) => ({...cur, username: value })), + }, + { + label: 'Password', + type: 'password', + stateHandler: (value, setter) => setter((curr) => ({...cur, password: value })), + }, + { + label: 'SASL Mechanism', + stateHandler: (value, setter) => setter((curr) => ({...cur, sasl: value })), + helpfulLink: 'https://docs.redpanda.com/current/manage/security/authentication/#scram', + }, +]; + +export const blankKaSetting: S3Config = { + servers: [], + username: '', + password: '', + sasl: 'SCRAM-SHA-512', + disableTls: false, +}; diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index 4aa958756a..053bb9bc09 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -3,6 +3,7 @@ import { PeerConfig } from '@/app/dto/PeersDTO'; import GuideForDestinationSetup from '@/app/mirrors/create/cdc/guide'; import BigqueryForm from '@/components/PeerForms/BigqueryConfig'; import ClickhouseForm from '@/components/PeerForms/ClickhouseConfig'; +import KafkaForm from '@/components/PeerForms/KafkaForm'; import PostgresForm from '@/components/PeerForms/PostgresForm'; import S3Form from '@/components/PeerForms/S3Form'; import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; @@ -81,6 +82,8 @@ export default function CreateConfig({ ); case 'S3': return ; + case 'KAFKA': + return ; default: return <>; } diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 211acd69a7..447cd2d6fe 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -278,6 +278,17 @@ export const chSchema = z.object({ region: z .string({ invalid_type_error: 'Region must be a string' }) .optional(), + disableTls: z.boolean(), +}); + +export const kaSchema = z.object({ + servers: z.array(z.string()), + username: z.string(), + password: z.string(), + sasl: z + .union([z.literal('SCRAM-SHA-256'), z.literal('SCRAM-SHA-512')]) + .optional(), + disableTls: z.boolean(), }); const urlSchema = z diff --git a/ui/components/PeerComponent.tsx b/ui/components/PeerComponent.tsx index 66851820ed..8505655c72 100644 --- a/ui/components/PeerComponent.tsx +++ b/ui/components/PeerComponent.tsx @@ -32,6 +32,9 @@ export const DBTypeToImageMapping = (peerType: DBType | string) => { case DBType.EVENTHUB_GROUP: case DBType.EVENTHUB: return '/svgs/ms.svg'; + case DBType.KAFKA: + case 'KAFKA': + return '/svgs/kafka.svg'; default: return '/svgs/pg.svg'; } diff --git a/ui/components/PeerForms/KafkaConfig.tsx b/ui/components/PeerForms/KafkaConfig.tsx new file mode 100644 index 0000000000..f3c580f155 --- /dev/null +++ b/ui/components/PeerForms/KafkaConfig.tsx @@ -0,0 +1,77 @@ +'use client'; +import { PeerSetter } from '@/app/dto/PeersDTO'; +import { kaSetting } from '@/app/peers/create/[peerType]/helpers/ka'; +import { Label } from '@/lib/Label'; +import { RowWithRadiobutton, RowWithTextField } from '@/lib/Layout'; +import { RadioButton, RadioButtonGroup } from '@/lib/RadioButtonGroup'; +import { TextField } from '@/lib/TextField'; +import { Tooltip } from '@/lib/Tooltip'; +import { useEffect, useState } from 'react'; +import { InfoPopover } from '../InfoPopover'; + +interface KafkaProps { + setter: PeerSetter; +} +const KafkaForm = ({ setter }: KafkaProps) => { + return ( +
+ + {kaSetting.map((setting, index) => { + if (displayCondition(setting.label)) + return ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ ) => + setting.stateHandler(e.target.value, setter) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ); + })} +
+ ); +}; + +export default KafkaForm; diff --git a/ui/components/PeerTypeComponent.tsx b/ui/components/PeerTypeComponent.tsx index d315499a6d..2db4be9dd2 100644 --- a/ui/components/PeerTypeComponent.tsx +++ b/ui/components/PeerTypeComponent.tsx @@ -24,6 +24,8 @@ export const DBTypeToGoodText = (ptype: DBType) => { return 'MongoDB'; case DBType.CLICKHOUSE: return 'Clickhouse'; + case DBType.KAFKA: + return 'Kafka'; case DBType.UNRECOGNIZED: return 'Unrecognised'; } diff --git a/ui/components/SelectSource.tsx b/ui/components/SelectSource.tsx index e77d1b5e4d..eca7c37e61 100644 --- a/ui/components/SelectSource.tsx +++ b/ui/components/SelectSource.tsx @@ -34,7 +34,8 @@ export default function SelectSource({ value === 'SNOWFLAKE' || value === 'BIGQUERY' || value === 'S3' || - value === 'CLICKHOUSE') + value === 'CLICKHOUSE' || + value === 'KAFKA') ) .map((value) => ({ label: value, value })); diff --git a/ui/public/svgs/kafka.svg b/ui/public/svgs/kafka.svg new file mode 100644 index 0000000000..305d876447 --- /dev/null +++ b/ui/public/svgs/kafka.svg @@ -0,0 +1 @@ + From 6b6a8f948afe2d8e820b24c2d222d40e90c740e6 Mon Sep 17 00:00:00 2001 From: Amogh-Bharadwaj Date: Mon, 11 Mar 2024 13:40:16 +0530 Subject: [PATCH 02/31] cleanup UI, add missing parts --- ui/app/api/peers/route.ts | 7 + ui/app/dto/PeersDTO.ts | 4 +- ui/app/peers/create/[peerType]/helpers/ka.ts | 30 +++- ui/app/peers/create/[peerType]/page.tsx | 2 +- ui/app/peers/create/[peerType]/schema.ts | 18 ++- ui/components/PeerForms/KafkaConfig.tsx | 159 ++++++++++++------- 6 files changed, 148 insertions(+), 72 deletions(-) diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index 5d43442bd8..0691212af8 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -10,6 +10,7 @@ import { BigqueryConfig, ClickhouseConfig, DBType, + KafkaConfig, Peer, PostgresConfig, S3Config, @@ -63,6 +64,12 @@ const constructPeer = ( type: DBType.S3, s3Config: config as S3Config, }; + case 'KAFKA': + return { + name, + type: DBType.KAFKA, + kafkaConfig: config as KafkaConfig, + }; default: return; } diff --git a/ui/app/dto/PeersDTO.ts b/ui/app/dto/PeersDTO.ts index bbf5fbccde..3e7f5c0b41 100644 --- a/ui/app/dto/PeersDTO.ts +++ b/ui/app/dto/PeersDTO.ts @@ -1,6 +1,7 @@ import { BigqueryConfig, ClickhouseConfig, + KafkaConfig, PostgresConfig, S3Config, SnowflakeConfig, @@ -43,7 +44,8 @@ export type PeerConfig = | SnowflakeConfig | BigqueryConfig | ClickhouseConfig - | S3Config; + | S3Config + | KafkaConfig; export type CatalogPeer = { id: number; name: string; diff --git a/ui/app/peers/create/[peerType]/helpers/ka.ts b/ui/app/peers/create/[peerType]/helpers/ka.ts index 7aa58e5642..8f542a9f9b 100644 --- a/ui/app/peers/create/[peerType]/helpers/ka.ts +++ b/ui/app/peers/create/[peerType]/helpers/ka.ts @@ -1,30 +1,44 @@ import { KafkaConfig } from '@/grpc_generated/peers'; -import {PeerSetting } from './common'; +import { PeerSetting } from './common'; export const kaSetting: PeerSetting[] = [ { label: 'Servers', - stateHandler: (value, setter) => setter((curr) => ({...cur, servers: value.split(',') })), + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, servers: (value as string).split(',') })), tips: 'Brokers', - helpfulLink: 'https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#SeedBrokers', + helpfulLink: + 'https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#SeedBrokers', }, { label: 'Username', - stateHandler: (value, setter) => setter((curr) => ({...cur, username: value })), + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, username: value as string })), }, { label: 'Password', type: 'password', - stateHandler: (value, setter) => setter((curr) => ({...cur, password: value })), + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, password: value as string })), }, { label: 'SASL Mechanism', - stateHandler: (value, setter) => setter((curr) => ({...cur, sasl: value })), - helpfulLink: 'https://docs.redpanda.com/current/manage/security/authentication/#scram', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, sasl: value as string })), + type: 'select', + helpfulLink: + 'https://docs.redpanda.com/current/manage/security/authentication/#scram', + }, + { + label: 'Disable TLS?', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, disableTls: value as boolean })), + type: 'switch', + tips: 'If you are using a non-TLS connection for Kafka server, check this box.', }, ]; -export const blankKaSetting: S3Config = { +export const blankKaSetting: KafkaConfig = { servers: [], username: '', password: '', diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index 053bb9bc09..bbb25b31ad 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -3,7 +3,7 @@ import { PeerConfig } from '@/app/dto/PeersDTO'; import GuideForDestinationSetup from '@/app/mirrors/create/cdc/guide'; import BigqueryForm from '@/components/PeerForms/BigqueryConfig'; import ClickhouseForm from '@/components/PeerForms/ClickhouseConfig'; -import KafkaForm from '@/components/PeerForms/KafkaForm'; +import KafkaForm from '@/components/PeerForms/KafkaConfig'; import PostgresForm from '@/components/PeerForms/PostgresForm'; import S3Form from '@/components/PeerForms/S3Form'; import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 447cd2d6fe..440a718cdb 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -282,13 +282,21 @@ export const chSchema = z.object({ }); export const kaSchema = z.object({ - servers: z.array(z.string()), - username: z.string(), - password: z.string(), + servers: z.array(z.string()).min(1, { message: 'Atleast 1 server required' }), + username: z + .string({ required_error: 'Username is required' }) + .min(1, { message: 'Username cannot be empty' }), + password: z + .string({ required_error: 'Password is required' }) + .min(1, { message: 'Password cannot be empty' }), sasl: z - .union([z.literal('SCRAM-SHA-256'), z.literal('SCRAM-SHA-512')]) + .union([z.literal('SCRAM-SHA-256'), z.literal('SCRAM-SHA-512')], { + errorMap: () => ({ + message: 'Either SCRAM-SHA-256 or SCRAM-SHA-512 is required.', + }), + }) .optional(), - disableTls: z.boolean(), + disableTls: z.boolean().optional(), }); const urlSchema = z diff --git a/ui/components/PeerForms/KafkaConfig.tsx b/ui/components/PeerForms/KafkaConfig.tsx index f3c580f155..e9acc40337 100644 --- a/ui/components/PeerForms/KafkaConfig.tsx +++ b/ui/components/PeerForms/KafkaConfig.tsx @@ -1,74 +1,119 @@ 'use client'; import { PeerSetter } from '@/app/dto/PeersDTO'; import { kaSetting } from '@/app/peers/create/[peerType]/helpers/ka'; +import SelectTheme from '@/app/styles/select'; import { Label } from '@/lib/Label'; -import { RowWithRadiobutton, RowWithTextField } from '@/lib/Layout'; -import { RadioButton, RadioButtonGroup } from '@/lib/RadioButtonGroup'; +import { RowWithSelect, RowWithSwitch, RowWithTextField } from '@/lib/Layout'; +import { Switch } from '@/lib/Switch/Switch'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; -import { useEffect, useState } from 'react'; +import ReactSelect from 'react-select'; import { InfoPopover } from '../InfoPopover'; - interface KafkaProps { setter: PeerSetter; } + +const saslOptions = [ + { value: 'SCRAM-SHA-256', label: 'SCRAM-SHA-256' }, + { value: 'SCRAM-SHA-512', label: 'SCRAM-SHA-512' }, +]; + const KafkaForm = ({ setter }: KafkaProps) => { return (
- + {kaSetting.map((setting, index) => { - if (displayCondition(setting.label)) - return ( - - {setting.label}{' '} - {!setting.optional && ( - - - - )} - - } - action={ -
- ) => - setting.stateHandler(e.target.value, setter) - } - /> - {setting.tips && ( - - )} -
- } - /> - ); + return setting.type === 'switch' ? ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ + setting.stateHandler(state, setter) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ) : setting.type === 'select' ? ( + SASL Mechanism} + action={ + + val && setting.stateHandler(val.value, setter) + } + options={saslOptions} + theme={SelectTheme} + /> + } + /> + ) : ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ ) => + setting.stateHandler(e.target.value, setter) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ); })}
); From 046ec57a3834777ec9b01dd01e6cb1963475d1b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 11 Mar 2024 12:39:36 +0000 Subject: [PATCH 03/31] Add script to ui, script is name, rely on loader to retrieve from catalog Flesh out qvalue types & add more types for non-primitive qvalue kinds BinaryArrayNew: back then I thought lua parameters were passed on the stack.. nope EndVector: need to clear nested bit PrependSlot: handle U64/I64 == number check On that note, implement __eq/__le/__lt for int64/uint64 Also implement general i64/u64/float64 methods for int64/uint64 & get rid of NI64/NU64 types, just use unwrapped integers Use ls.RaisePanic over panic Wire print to LogFlowInfo --- flow/activities/flowable.go | 1 + flow/cmd/handler.go | 11 +- flow/connectors/external_metadata/store.go | 7 + flow/connectors/kafka/kafka.go | 56 ++- flow/model/qrecord_test.go | 4 +- flow/model/qvalue/qvalue.go | 5 +- flow/pua/flatbuffers.go | 47 +- flow/pua/flatbuffers_binaryarray.go | 12 +- flow/pua/flatbuffers_builder.go | 89 ++-- flow/pua/flatbuffers_numtypes.go | 31 +- flow/pua/peerdb.go | 437 ++++++++++++++++-- nexus/analyzer/src/lib.rs | 10 +- .../{V21__scripts.sql => V23__scripts.sql} | 2 +- nexus/catalog/src/lib.rs | 9 +- nexus/flow-rs/src/grpc.rs | 3 +- nexus/pt/src/flow_model.rs | 3 +- protos/flow.proto | 2 + ui/app/api/peers/info/[peerName]/route.ts | 14 +- ui/app/mirrors/create/helpers/cdc.ts | 10 + ui/app/mirrors/create/helpers/common.ts | 1 + ui/app/peers/create/[peerType]/helpers/ka.ts | 2 +- ui/app/peers/create/[peerType]/schema.ts | 17 +- ui/components/PeerForms/KafkaConfig.tsx | 1 + 23 files changed, 603 insertions(+), 171 deletions(-) rename nexus/catalog/migrations/{V21__scripts.sql => V23__scripts.sql} (80%) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 52ab1a8ca7..3d6e7815b7 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -400,6 +400,7 @@ func (a *FlowableActivity) SyncFlow( FlowJobName: flowName, TableMappings: options.TableMappings, StagingPath: config.CdcStagingPath, + Script: config.Script, }) if err != nil { a.Alerter.LogFlowError(ctx, flowName, err) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 554e177dc1..f0264608bb 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -533,9 +533,7 @@ func (h *FlowRequestHandler) CreatePeer( return wrongConfigResponse, nil } pgConfig := pgConfigObject.PostgresConfig - encodedConfig, encodingErr = proto.Marshal(pgConfig) - case protos.DBType_SNOWFLAKE: sfConfigObject, ok := config.(*protos.Peer_SnowflakeConfig) if !ok { @@ -566,13 +564,18 @@ func (h *FlowRequestHandler) CreatePeer( encodedConfig, encodingErr = proto.Marshal(s3Config) case protos.DBType_CLICKHOUSE: chConfigObject, ok := config.(*protos.Peer_ClickhouseConfig) - if !ok { return wrongConfigResponse, nil } - chConfig := chConfigObject.ClickhouseConfig encodedConfig, encodingErr = proto.Marshal(chConfig) + case protos.DBType_KAFKA: + kaConfigObject, ok := config.(*protos.Peer_KafkaConfig) + if !ok { + return wrongConfigResponse, nil + } + kaConfig := kaConfigObject.KafkaConfig + encodedConfig, encodingErr = proto.Marshal(kaConfig) default: return wrongConfigResponse, nil } diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index 5fffec0f69..bbe986b497 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -55,6 +55,13 @@ func (p *PostgresMetadataStore) Ping(ctx context.Context) error { return nil } +func (p *PostgresMetadataStore) LogFlowInfo(ctx context.Context, flowName string, info string) error { + _, err := p.pool.Exec(ctx, + "INSERT INTO peerdb_stats.flow_errors(flow_name,error_message,error_type) VALUES($1,$2,$3)", + flowName, info, "info") + return err +} + func (p *PostgresMetadataStore) FetchLastOffset(ctx context.Context, jobName string) (int64, error) { row := p.pool.QueryRow(ctx, `SELECT last_offset FROM `+ diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 3ccb17d31f..920d14c872 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -1,15 +1,16 @@ package connkafka import ( - "bytes" "context" "crypto/tls" "errors" "fmt" "log/slog" + "strings" "sync" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/sasl/plain" "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/plugin/kslog" "github.com/yuin/gopher-lua" @@ -44,11 +45,15 @@ func NewKafkaConnector( optionalOpts = append(optionalOpts, kgo.DialTLSConfig(&tls.Config{MinVersion: tls.VersionTLS13})) } if config.Username != "" { - auth := scram.Auth{User: config.Username, Pass: config.Password} switch config.Sasl { + case "PLAIN": + auth := plain.Auth{User: config.Username, Pass: config.Password} + optionalOpts = append(optionalOpts, kgo.SASL(auth.AsMechanism())) case "SCRAM-SHA-256": + auth := scram.Auth{User: config.Username, Pass: config.Password} optionalOpts = append(optionalOpts, kgo.SASL(auth.AsSha256Mechanism())) case "SCRAM-SHA-512": + auth := scram.Auth{User: config.Username, Pass: config.Password} optionalOpts = append(optionalOpts, kgo.SASL(auth.AsSha512Mechanism())) default: return nil, fmt.Errorf("unsupported SASL mechanism: %s", config.Sasl) @@ -59,9 +64,15 @@ func NewKafkaConnector( return nil, fmt.Errorf("failed to create kafka client: %w", err) } + pgMetadata, err := metadataStore.NewPostgresMetadataStore(ctx) + if err != nil { + return nil, err + } + return &KafkaConnector{ - client: client, - logger: logger.LoggerFromCtx(ctx), + client: client, + pgMetadata: pgMetadata, + logger: logger.LoggerFromCtx(ctx), }, nil } @@ -101,7 +112,6 @@ func (c *KafkaConnector) SetupMetadataTables(_ context.Context) error { } func (c *KafkaConnector) ReplayTableSchemaDeltas(_ context.Context, flowJobName string, schemaDeltas []*protos.TableSchemaDelta) error { - c.logger.Info("ReplayTableSchemaDeltas for event hub is a no-op") return nil } @@ -110,10 +120,7 @@ func (c *KafkaConnector) SyncFlowCleanup(ctx context.Context, jobName string) er } func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - err := c.client.BeginTransaction() - if err != nil { - return nil, err - } + // TODO BeginTransaction if transactional var wg sync.WaitGroup wgCtx, wgErr := context.WithCancelCause(ctx) @@ -152,15 +159,29 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords } ls.PreloadModule("flatbuffers", pua.FlatBuffers_Loader) pua.RegisterTypes(ls) - err := ls.DoString(req.Script) + ls.Env.RawSetString("print", ls.NewFunction(func(ls *lua.LState) int { + top := ls.GetTop() + ss := make([]string, top) + for i := range top { + ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + } + _ = c.pgMetadata.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) + return 0 + })) + err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(req.Script)) if err != nil { - return nil, fmt.Errorf("error while executing script: %w", err) + return nil, fmt.Errorf("error loading script %s: %w", req.Script, err) + } + err = ls.PCall(0, 0, nil) + if err != nil { + return nil, fmt.Errorf("error executing script %s: %w", req.Script, err) } var ok bool - fn, ok = ls.GetGlobal("onRow").(*lua.LFunction) + lfn := ls.Env.RawGetString("onRecord") + fn, ok = lfn.(*lua.LFunction) if !ok { - return nil, errors.New("script should define `onRow` function") + return nil, fmt.Errorf("script should define `onRecord` as function, not %s", lfn) } } else { return nil, errors.New("kafka mirror must have script") @@ -184,14 +205,13 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("script returned non-nil non-string: %v", value) } wg.Add(1) - c.client.Produce(wgCtx, &kgo.Record{Topic: topic, Value: bytes.Clone([]byte(lstr))}, produceCb) + c.client.Produce(wgCtx, &kgo.Record{Topic: topic, Value: []byte(lstr)}, produceCb) numRecords += 1 tableNameRowsMapping[topic] += 1 } } - // TODO handle waitChan := make(chan struct{}) go func() { wg.Wait() @@ -207,12 +227,10 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("could not flush transaction: %w", err) } - if err := c.client.EndTransaction(ctx, kgo.TryCommit); err != nil { - return nil, fmt.Errorf("could not commit transaction: %w", err) - } + // TODO EndTransaction if transactional lastCheckpoint := req.Records.GetLastCheckpoint() - err = c.pgMetadata.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) + err := c.pgMetadata.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) if err != nil { return nil, err } diff --git a/flow/model/qrecord_test.go b/flow/model/qrecord_test.go index 12b59c67bb..e6c769fd69 100644 --- a/flow/model/qrecord_test.go +++ b/flow/model/qrecord_test.go @@ -12,8 +12,8 @@ import ( ) func TestEquals(t *testing.T) { - uuidVal1, _ := uuid.NewRandom() - uuidVal2, _ := uuid.NewRandom() + uuidVal1 := uuid.New() + uuidVal2 := uuid.New() tests := []struct { name string diff --git a/flow/model/qvalue/qvalue.go b/flow/model/qvalue/qvalue.go index 4a495a31cc..bedc3decec 100644 --- a/flow/model/qvalue/qvalue.go +++ b/flow/model/qvalue/qvalue.go @@ -606,10 +606,7 @@ func getUUID(v interface{}) (uuid.UUID, bool) { return parsed, true } case [16]byte: - parsed, err := uuid.FromBytes(value[:]) - if err == nil { - return parsed, true - } + return uuid.UUID(value), true } return uuid.UUID{}, false diff --git a/flow/pua/flatbuffers.go b/flow/pua/flatbuffers.go index a6c8c7a23a..c27431007a 100644 --- a/flow/pua/flatbuffers.go +++ b/flow/pua/flatbuffers.go @@ -4,25 +4,6 @@ import ( "github.com/yuin/gopher-lua" ) -/* -local m = {} - -m.Builder = require("flatbuffers.builder").New -m.N = require("flatbuffers.numTypes") -m.view = require("flatbuffers.view") -m.binaryArray = require("flatbuffers.binaryarray") - -return m -*/ - -func requireHelper(ls *lua.LState, m *lua.LTable, require lua.LValue, name string, path string) { - ls.Push(require) - ls.Push(lua.LString(path)) - ls.Call(1, 1) - ls.SetField(m, name, ls.Get(-1)) - ls.Pop(1) -} - func FlatBuffers_Loader(ls *lua.LState) int { ls.PreloadModule("flatbuffers.binaryarray", FlatBuffers_BinaryArray_Loader) ls.PreloadModule("flatbuffers.builder", FlatBuffers_Builder_Loader) @@ -30,17 +11,27 @@ func FlatBuffers_Loader(ls *lua.LState) int { ls.PreloadModule("flatbuffers.view", FlatBuffers_View_Loader) m := ls.NewTable() - require := ls.GetGlobal("require") - ls.Push(require) - ls.Push(lua.LString("flatbuffers.builder")) - ls.Call(1, 1) - builder := ls.GetTable(ls.Get(-1), lua.LString("New")) - ls.SetField(m, "builder", builder) + + ls.Push(ls.NewFunction(FlatBuffers_N_Loader)) + ls.Call(0, 1) + m.RawSetString("N", ls.Get(-1)) ls.Pop(1) - requireHelper(ls, m, require, "N", "flatbuffers.numTypes") - requireHelper(ls, m, require, "view", "flatbuffers.view") - requireHelper(ls, m, require, "binaryArray", "flatbuffers.binaryarray") + ls.Push(ls.NewFunction(FlatBuffers_View_Loader)) + ls.Call(0, 1) + m.RawSetString("view", ls.Get(-1)) + ls.Pop(1) + + ls.Push(ls.NewFunction(FlatBuffers_BinaryArray_Loader)) + ls.Call(0, 1) + m.RawSetString("binaryArray", ls.Get(-1)) + ls.Pop(1) + + ls.Push(ls.NewFunction(FlatBuffers_Builder_Loader)) + ls.Call(0, 1) + builder := ls.GetField(ls.Get(-1), "New") + m.RawSetString("Builder", builder) + ls.Pop(1) ls.Push(m) return 1 diff --git a/flow/pua/flatbuffers_binaryarray.go b/flow/pua/flatbuffers_binaryarray.go index e296c871bf..635e8f7f55 100644 --- a/flow/pua/flatbuffers_binaryarray.go +++ b/flow/pua/flatbuffers_binaryarray.go @@ -27,7 +27,7 @@ func FlatBuffers_BinaryArray_Loader(ls *lua.LState) int { } func BinaryArrayNew(ls *lua.LState) int { - lval := ls.Get(-1) + lval := ls.Get(1) var ba BinaryArray switch val := lval.(type) { case lua.LString: @@ -85,17 +85,13 @@ func BinaryArraySlice(ls *lua.LState) int { return 1 } -func (ba *BinaryArray) Grow(newsize int) { - newdata := make([]byte, newsize) - copy(newdata[newsize-len(ba.data):], ba.data) - ba.data = newdata -} - func BinaryArrayGrow(ls *lua.LState) int { baud, ba := LuaBinaryArray.Check(ls, 1) newsize := int(ls.CheckNumber(2)) if newsize > len(ba.data) { - ba.Grow(newsize) + newdata := make([]byte, newsize) + copy(newdata[newsize-len(ba.data):], ba.data) + ba.data = newdata baud.Value = ba } return 0 diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers_builder.go index 6abf8f55fc..0c024446c9 100644 --- a/flow/pua/flatbuffers_builder.go +++ b/flow/pua/flatbuffers_builder.go @@ -1,6 +1,8 @@ package pua import ( + "slices" + "github.com/yuin/gopher-lua" ) @@ -17,10 +19,12 @@ type Builder struct { minalign uint8 } -func (b *Builder) EndVector(vectorSize int) int { +func (b *Builder) EndVector(ls *lua.LState, vectorSize int) int { if !b.nested { - panic("EndVector called outside nested context") + ls.RaiseError("EndVector called outside nested context") + return 0 } + b.nested = false b.PlaceU64(uint64(vectorSize), uint32n) return b.Offset() } @@ -51,17 +55,16 @@ func (b *Builder) Prep(width uint8, additional int) { b.minalign = width } k := len(b.ba.data) - b.head + additional - alignsize := (^k + 1) & int(width-1) - desiredSize := alignsize + int(width) + additional - - for b.head < desiredSize { - oldBufSize := len(b.ba.data) - newBufSize := oldBufSize + 1 - for newBufSize < desiredSize { - newBufSize *= 2 - } - b.ba.Grow(newBufSize) - b.head += len(b.ba.data) - oldBufSize + alignsize := -k & int(width-1) + + space := alignsize + int(width) + additional + + if b.head < space { + oldlen := len(b.ba.data) + newdata := slices.Grow(b.ba.data, space) + newdata = newdata[:cap(newdata)] + copy(newdata[:oldlen], newdata[len(newdata)-oldlen:]) + b.head += len(newdata) - oldlen } b.Pad(alignsize) @@ -78,10 +81,26 @@ func (b *Builder) PrependU64(n N, x uint64) { } func (b *Builder) PrependSlot(ls *lua.LState, n N, slotnum int, x lua.LValue, d lua.LValue) { - // TODO implement __eq for U64/I64 if !ls.Equal(x, d) { + if xud, ok := x.(*lua.LUserData); ok { + // Need to check int64/number because flatbuffers passes default as 0 + // but Lua only calls __eq when both operands are same type + if dn, ok := d.(lua.LNumber); ok { + switch xv := xud.Value.(type) { + case int64: + if xv == int64(dn) { + return + } + case uint64: + if xv == uint64(dn) { + return + } + } + } + } + b.Prepend(ls, n, x) - b.Slot(slotnum) + b.Slot(ls, slotnum) } } @@ -108,12 +127,13 @@ func (b *Builder) PrependVOffsetT(off uint16) { b.PlaceU64(uint64(off), uint16n) } -func (b *Builder) Slot(slotnum int) { +func (b *Builder) Slot(ls *lua.LState, slotnum int) { if !b.nested { - panic("Slot called outside nested context") + ls.RaiseError("Slot called outside nested context") + return } - for slotnum < len(b.currentVT) { - b.currentVT = append(b.currentVT, 0) + if slotnum >= len(b.currentVT) { + b.currentVT = slices.Grow(b.currentVT, slotnum-len(b.currentVT)+1) } b.currentVT[slotnum] = b.Offset() } @@ -141,7 +161,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { } var existingVtable int - for i := len(b.ba.data) - 1; i >= 0; i-- { + for i := len(b.vtables) - 1; i >= 0; i -= 1 { vt2Offset := b.vtables[i] vt2Start := len(b.ba.data) - vt2Offset vt2Len := uint16n.UnpackU64(b.ba.data[vt2Start:]) @@ -157,7 +177,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { } if existingVtable == 0 { - for i := len(b.currentVT) - 1; i >= 0; i-- { + for i := len(b.currentVT) - 1; i >= 0; i -= 1 { var off uint16 if b.currentVT[i] != 0 { off = uint16(objectOffset - b.currentVT[i]) @@ -191,9 +211,9 @@ var LuaBuilder = LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} func FlatBuffers_Builder_Loader(ls *lua.LState) int { m := ls.NewTable() - ls.SetField(m, "New", ls.NewFunction(BuilderNew)) + m.RawSetString("New", ls.NewFunction(BuilderNew)) - mt := LuaBinaryArray.NewMetatable(ls) + mt := LuaBuilder.NewMetatable(ls) index := ls.SetFuncs(ls.NewTable(), map[string]lua.LGFunction{ "Clear": BuilderClear, "Output": BuilderOutput, @@ -300,11 +320,11 @@ func BuilderStartObject(ls *lua.LState) int { ls.RaiseError("StartObject called inside nested context") return 0 } + b.nested = true numFields := int(ls.CheckNumber(2)) b.currentVT = make([]int, numFields) b.objectEnd = b.Offset() - b.nested = true return 0 } @@ -317,8 +337,10 @@ func BuilderWriteVtable(ls *lua.LState) int { func BuilderEndObject(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) if !b.nested { - panic("EndObject called outside nested context") + ls.RaiseError("EndObject called outside nested context") + return 0 } + b.nested = false ls.Push(lua.LNumber(b.WriteVtable(ls))) return 1 } @@ -380,12 +402,8 @@ func BuilderStartVector(ls *lua.LState) int { func BuilderEndVector(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) - if !b.nested { - ls.RaiseError("EndVector called outside nested context") - } - b.nested = false - b.PlaceU64(uint64(ls.CheckNumber(2)), uint32n) - ls.Push(lua.LNumber(b.Offset())) + size := int(ls.CheckNumber(2)) + ls.Push(lua.LNumber(b.EndVector(ls, size))) return 1 } @@ -404,13 +422,14 @@ func BuilderCreateString(ls *lua.LState) int { b.head -= lens copy(b.ba.data[b.head:], s) - return b.EndVector(lens) + ls.Push(lua.LNumber(b.EndVector(ls, lens))) + return 1 } func BuilderSlot(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) slotnum := int(ls.CheckNumber(2)) - b.Slot(slotnum) + b.Slot(ls, slotnum) return 0 } @@ -516,7 +535,7 @@ func BuilderPrependStructSlot(ls *lua.LState) int { if x != b.Offset() { ls.RaiseError("Tried to write a Struct at an Offset that is different from the current Offset of the Builder.") } else { - b.Slot(int(ls.CheckNumber(2))) + b.Slot(ls, int(ls.CheckNumber(2))) } } return 0 @@ -528,7 +547,7 @@ func BuilderPrependUOffsetTRelativeSlot(ls *lua.LState) int { d := int(ls.CheckNumber(4)) if x != d { b.PrependOffsetTRelative(ls, x, uint32n) - b.Slot(int(ls.CheckNumber(2))) + b.Slot(ls, int(ls.CheckNumber(2))) } return 0 } diff --git a/flow/pua/flatbuffers_numtypes.go b/flow/pua/flatbuffers_numtypes.go index b174bee873..32a23fc205 100644 --- a/flow/pua/flatbuffers_numtypes.go +++ b/flow/pua/flatbuffers_numtypes.go @@ -59,10 +59,10 @@ func (n *N) Pack(ls *lua.LState, buf []byte, val lua.LValue) { switch lv := val.(type) { case *lua.LUserData: switch v := lv.Value.(type) { - case NI64: - n.PackU64(buf, uint64(v.val)) - case NU64: - n.PackU64(buf, v.val) + case int64: + n.PackU64(buf, uint64(v)) + case uint64: + n.PackU64(buf, v) default: n.PackU64(buf, 0) } @@ -88,10 +88,10 @@ func (n *N) Pack(ls *lua.LState, buf []byte, val lua.LValue) { switch lv := val.(type) { case *lua.LUserData: switch v := lv.Value.(type) { - case NI64: - n.PackU64(buf, math.Float64bits(float64(v.val))) - case NU64: - n.PackU64(buf, math.Float64bits(float64(v.val))) + case int64: + n.PackU64(buf, math.Float64bits(float64(v))) + case uint64: + n.PackU64(buf, math.Float64bits(float64(v))) default: n.PackU64(buf, 0) } @@ -146,9 +146,9 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { case 8: u64 := binary.LittleEndian.Uint64(buf) if n.signed { - return LuaNI64.New(ls, NI64{int64(u64)}) + return LuaI64.New(ls, int64(u64)) } else { - return LuaNU64.New(ls, NU64{u64}) + return LuaU64.New(ls, u64) } } case tyfloat: @@ -165,16 +165,7 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { panic("invalid numeric metatype") } -type ( - NI64 struct{ val int64 } - NU64 struct{ val uint64 } -) - -var ( - LuaN = LuaUserDataType[N]{Name: "flatbuffers_n"} - LuaNI64 = LuaUserDataType[NI64]{Name: "flatbuffers_i64"} - LuaNU64 = LuaUserDataType[NU64]{Name: "flatbuffers_u64"} -) +var LuaN = LuaUserDataType[N]{Name: "flatbuffers_n"} func FlatBuffers_N_Loader(ls *lua.LState) int { m := ls.NewTable() diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index a42eb8dd48..8c1cefaf73 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -3,10 +3,14 @@ package pua import ( "bytes" "fmt" + "math/big" "reflect" + "strconv" "strings" "time" + "github.com/google/uuid" + "github.com/jackc/pgx/v5" "github.com/yuin/gopher-lua" "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" @@ -18,30 +22,66 @@ var ( LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} LuaQValue = LuaUserDataType[qvalue.QValue]{Name: "peerdb_value"} + LuaI64 = LuaUserDataType[int64]{Name: "flatbuffers_i64"} + LuaU64 = LuaUserDataType[uint64]{Name: "flatbuffers_u64"} + LuaTime = LuaUserDataType[time.Time]{Name: "peerdb_time"} + LuaUuid = LuaUserDataType[uuid.UUID]{Name: "peerdb_uuid"} + LuaBigInt = LuaUserDataType[*big.Int]{Name: "peerdb_bigint"} + LuaRat = LuaUserDataType[*big.Rat]{Name: "peerdb_bigrat"} ) func RegisterTypes(ls *lua.LState) { // gopher-lua provides 2 loaders {preload, file} // overwrite file loader with one retrieving scripts from database - loaders := ls.GetField(ls.Get(lua.RegistryIndex), "_LOADERS").(*lua.LTable) - ls.RawSetInt(loaders, 2, ls.NewFunction(LoadPeerdbScript)) + loaders := ls.G.Registry.RawGetString("_LOADERS").(*lua.LTable) + loaders.RawSetInt(2, ls.NewFunction(LoadPeerdbScript)) mt := LuaRecord.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(LuaRecordIndex)) + mt.RawSetString("__index", ls.NewFunction(LuaRecordIndex)) mt = LuaRow.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(LuaRowIndex)) - ls.SetField(mt, "__len", ls.NewFunction(LuaRowLen)) + mt.RawSetString("__index", ls.NewFunction(LuaRowIndex)) + mt.RawSetString("__len", ls.NewFunction(LuaRowLen)) mt = LuaQValue.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(LuaQValueIndex)) - ls.SetField(mt, "__len", ls.NewFunction(LuaQValueLen)) + mt.RawSetString("__index", ls.NewFunction(LuaQValueIndex)) + mt.RawSetString("__len", ls.NewFunction(LuaQValueLen)) + mt.RawSetString("__tostring", ls.NewFunction(LuaQValueString)) + + mt = LuaUuid.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) + mt.RawSetString("__tostring", ls.NewFunction(LuaUuidString)) + + mt = LuaI64.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaI64Index)) + mt.RawSetString("__tostring", ls.NewFunction(LuaI64String)) + mt.RawSetString("__eq", ls.NewFunction(Lua64Eq)) + mt.RawSetString("__le", ls.NewFunction(Lua64Le)) + mt.RawSetString("__lt", ls.NewFunction(Lua64Lt)) + + mt = LuaU64.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaU64Index)) + mt.RawSetString("__tostring", ls.NewFunction(LuaU64String)) + mt.RawSetString("__eq", ls.NewFunction(Lua64Eq)) + mt.RawSetString("__le", ls.NewFunction(Lua64Le)) + mt.RawSetString("__lt", ls.NewFunction(Lua64Lt)) + + mt = LuaTime.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaTimeIndex)) + + mt = LuaBigInt.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaBigIntIndex)) + mt.RawSetString("__len", ls.NewFunction(LuaBigIntLen)) + + mt = LuaRat.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaRatIndex)) peerdb := ls.NewTable() - ls.SetField(peerdb, "RowToJSON", ls.NewFunction(LuaRowToJSON)) - ls.SetField(peerdb, "RowColumns", ls.NewFunction(LuaRowColumns)) - ls.SetField(peerdb, "UnixNow", ls.NewFunction(LuaUnixNow)) - ls.SetGlobal("peerdb", peerdb) + peerdb.RawSetString("RowToJSON", ls.NewFunction(LuaRowToJSON)) + peerdb.RawSetString("RowColumns", ls.NewFunction(LuaRowColumns)) + peerdb.RawSetString("Now", ls.NewFunction(LuaNow)) + peerdb.RawSetString("UUID", ls.NewFunction(LuaUUID)) + ls.Env.RawSetString("peerdb", peerdb) } func LoadPeerdbScript(ls *lua.LState) int { @@ -52,15 +92,23 @@ func LoadPeerdbScript(ls *lua.LState) int { ls.RaiseError("Connection failed loading %s: %s", name, err.Error()) return 0 } + var source []byte err = pool.QueryRow(ctx, "select source from scripts where lang = 'lua' and name = $1", name).Scan(&source) - if err == nil { - fn, err := ls.Load(bytes.NewReader(source), name) - if err != nil { - ls.RaiseError(err.Error()) + if err != nil { + if err == pgx.ErrNoRows { + ls.Push(lua.LString("Could not find script " + name)) + return 1 } - ls.Push(fn) + ls.RaiseError("Failed to load script %s: %s", name, err.Error()) + return 0 + } + + fn, err := ls.Load(bytes.NewReader(source), name) + if err != nil { + ls.RaiseError(err.Error()) } + ls.Push(fn) return 1 } @@ -98,7 +146,7 @@ func LuaRowColumns(ls *lua.LState) int { _, row := LuaRow.Check(ls, 1) tbl := ls.CreateTable(len(row.ColToValIdx), 0) for col, idx := range row.ColToValIdx { - ls.RawSetInt(tbl, idx, lua.LString(col)) + tbl.RawSetInt(idx, lua.LString(col)) } ls.Push(tbl) return 1 @@ -108,18 +156,16 @@ func LuaRecordIndex(ls *lua.LState) int { record, key := LuaRecord.StartIndex(ls) switch key { case "kind": - var tyname string switch record.(type) { case *model.InsertRecord: - tyname = "insert" + ls.Push(lua.LString("insert")) case *model.UpdateRecord: - tyname = "update" + ls.Push(lua.LString("update")) case *model.DeleteRecord: - tyname = "delete" + ls.Push(lua.LString("delete")) case *model.RelationRecord: - tyname = "relation" + ls.Push(lua.LString("relation")) } - ls.Push(lua.LString(tyname)) case "row": items := record.GetItems() if items != nil { @@ -154,7 +200,7 @@ func LuaRecordIndex(ls *lua.LState) int { ls.Push(lua.LNil) } case "checkpoint": - ls.Push(LuaNI64.New(ls, NI64{record.GetCheckpointID()})) + ls.Push(LuaI64.New(ls, record.GetCheckpointID())) case "target": ls.Push(lua.LString(record.GetDestinationTableName())) case "source": @@ -165,17 +211,97 @@ func LuaRecordIndex(ls *lua.LState) int { return 1 } +func qvToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) { + tbl := ls.CreateTable(len(s), 0) + for idx, val := range s { + tbl.RawSetInt(idx, f(val)) + } + ls.Push(tbl) +} + func LuaQValueIndex(ls *lua.LState) int { qv, key := LuaQValue.StartIndex(ls) switch key { case "kind": ls.Push(lua.LString(qv.Kind)) + case "value": + switch v := qv.Value.(type) { + case nil: + ls.Push(lua.LNil) + case bool: + ls.Push(lua.LBool(v)) + case uint8: + if qv.Kind == qvalue.QValueKindQChar { + ls.Push(lua.LString(rune(v))) + } else { + ls.Push(lua.LNumber(v)) + } + case int16: + ls.Push(lua.LNumber(v)) + case int32: + ls.Push(lua.LNumber(v)) + case int64: + ls.Push(LuaI64.New(ls, v)) + case float32: + ls.Push(lua.LNumber(v)) + case float64: + ls.Push(lua.LNumber(v)) + case string: + if qv.Kind == qvalue.QValueKindUUID { + u, err := uuid.Parse(v) + if err != nil { + ls.Push(LuaUuid.New(ls, u)) + } else { + ls.Push(lua.LString(v)) + } + } else { + ls.Push(lua.LString(v)) + } + case time.Time: + ls.Push(LuaTime.New(ls, v)) + case *big.Rat: + ls.Push(LuaRat.New(ls, v)) + case [16]byte: + ls.Push(LuaUuid.New(ls, uuid.UUID(v))) + case []byte: + ls.Push(lua.LString(v)) + case []float32: + qvToLTable(ls, v, func(f float32) lua.LValue { + return lua.LNumber(f) + }) + case []float64: + qvToLTable(ls, v, func(f float64) lua.LValue { + return lua.LNumber(f) + }) + case []int16: + qvToLTable(ls, v, func(f int16) lua.LValue { + return lua.LNumber(f) + }) + case []int32: + qvToLTable(ls, v, func(f int32) lua.LValue { + return lua.LNumber(f) + }) + case []int64: + qvToLTable(ls, v, func(x int64) lua.LValue { + return LuaI64.New(ls, x) + }) + case []string: + qvToLTable(ls, v, func(x string) lua.LValue { + return lua.LString(x) + }) + case []time.Time: + qvToLTable(ls, v, func(x time.Time) lua.LValue { + return LuaTime.New(ls, x) + }) + case []bool: + qvToLTable(ls, v, func(x bool) lua.LValue { + return lua.LBool(x) + }) + } case "int64": - ls.Push(LuaNI64.New(ls, NI64{reflect.ValueOf(qv.Value).Int()})) + ls.Push(LuaI64.New(ls, reflect.ValueOf(qv.Value).Int())) case "float64": ls.Push(lua.LNumber(reflect.ValueOf(qv.Value).Float())) - case "string": - ls.Push(lua.LString(fmt.Sprint(qv.Value))) default: return 0 } @@ -196,7 +322,260 @@ func LuaQValueLen(ls *lua.LState) int { return 0 } -func LuaUnixNow(ls *lua.LState) int { - ls.Push(lua.LNumber(float64(time.Now().UnixMilli()) / 1000.0)) +func LuaQValueString(ls *lua.LState) int { + qv := LuaQValue.StartMeta(ls) + ls.Push(lua.LString(fmt.Sprint(qv.Value))) + return 1 +} + +func LuaUuidIndex(ls *lua.LState) int { + _, val := LuaUuid.Check(ls, 1) + key := ls.CheckNumber(2) + ki := int(key) + if ki >= 0 && ki < 16 { + ls.Push(lua.LNumber(val[ki])) + return 1 + } + return 0 +} + +func LuaUuidString(ls *lua.LState) int { + val := LuaUuid.StartMeta(ls) + ls.Push(lua.LString(val.String())) + return 1 +} + +func LuaNow(ls *lua.LState) int { + ls.Push(LuaTime.New(ls, time.Now())) + return 1 +} + +func LuaUUID(ls *lua.LState) int { + ls.Push(LuaUuid.New(ls, uuid.New())) + return 1 +} + +func Lua64Eq(ls *lua.LState) int { + aud := ls.CheckUserData(1) + bud := ls.CheckUserData(2) + switch a := aud.Value.(type) { + case int64: + switch b := bud.Value.(type) { + case int64: + ls.Push(lua.LBool(a == b)) + case uint64: + if a < 0 { + ls.Push(lua.LFalse) + } else { + ls.Push(lua.LBool(uint64(a) == b)) + } + default: + return 0 + } + case uint64: + switch b := bud.Value.(type) { + case int64: + if b < 0 { + ls.Push(lua.LFalse) + } else { + ls.Push(lua.LBool(a == uint64(b))) + } + case uint64: + ls.Push(lua.LBool(a == b)) + default: + return 0 + } + default: + return 0 + } + return 1 +} + +func Lua64Le(ls *lua.LState) int { + aud := ls.CheckUserData(1) + bud := ls.CheckUserData(2) + switch a := aud.Value.(type) { + case int64: + switch b := bud.Value.(type) { + case int64: + ls.Push(lua.LBool(a <= b)) + case uint64: + if a < 0 { + ls.Push(lua.LTrue) + } else { + ls.Push(lua.LBool(uint64(a) <= b)) + } + default: + return 0 + } + case uint64: + switch b := bud.Value.(type) { + case int64: + if b < 0 { + ls.Push(lua.LFalse) + } else { + ls.Push(lua.LBool(a <= uint64(b))) + } + case uint64: + ls.Push(lua.LBool(a <= b)) + default: + return 0 + } + default: + return 0 + } + return 1 +} + +func Lua64Lt(ls *lua.LState) int { + aud := ls.CheckUserData(1) + bud := ls.CheckUserData(2) + switch a := aud.Value.(type) { + case int64: + switch b := bud.Value.(type) { + case int64: + ls.Push(lua.LBool(a < b)) + case uint64: + if a < 0 { + ls.Push(lua.LTrue) + } else { + ls.Push(lua.LBool(uint64(a) < b)) + } + default: + return 0 + } + case uint64: + switch b := bud.Value.(type) { + case int64: + if b < 0 { + ls.Push(lua.LTrue) + } else { + ls.Push(lua.LBool(a < uint64(b))) + } + case uint64: + ls.Push(lua.LBool(a < b)) + default: + return 0 + } + default: + return 0 + } + return 1 +} + +func LuaI64Index(ls *lua.LState) int { + i64ud, i64 := LuaI64.Check(ls, 1) + key := ls.CheckString(2) + switch key { + case "i64": + ls.Push(i64ud) + case "u64": + ls.Push(LuaU64.New(ls, uint64(i64))) + case "float64": + ls.Push(lua.LNumber(i64)) + default: + return 0 + } + return 1 +} + +func LuaU64Index(ls *lua.LState) int { + u64ud, u64 := LuaU64.Check(ls, 1) + key := ls.CheckString(2) + switch key { + case "i64": + ls.Push(LuaI64.New(ls, int64(u64))) + case "u64": + ls.Push(u64ud) + case "float64": + ls.Push(lua.LNumber(u64)) + default: + return 0 + } + return 1 +} + +func LuaI64String(ls *lua.LState) int { + i64 := LuaI64.StartMeta(ls) + ls.Push(lua.LString(strconv.FormatInt(i64, 10))) + return 1 +} + +func LuaU64String(ls *lua.LState) int { + u64 := LuaU64.StartMeta(ls) + ls.Push(lua.LString(strconv.FormatUint(u64, 10))) + return 1 +} + +func LuaTimeIndex(ls *lua.LState) int { + tm, key := LuaTime.StartIndex(ls) + switch key { + case "unix_nano": + ls.Push(LuaI64.New(ls, tm.UnixNano())) + case "unix_micro": + ls.Push(LuaI64.New(ls, tm.UnixMicro())) + case "unix_milli": + ls.Push(LuaI64.New(ls, tm.UnixMilli())) + case "unix": + ls.Push(LuaI64.New(ls, tm.Unix())) + case "year": + ls.Push(lua.LNumber(tm.Year())) + case "month": + ls.Push(lua.LNumber(tm.Month())) + case "day": + ls.Push(lua.LNumber(tm.Day())) + case "yearday": + ls.Push(lua.LNumber(tm.YearDay())) + case "hour": + ls.Push(lua.LNumber(tm.Hour())) + case "minute": + ls.Push(lua.LNumber(tm.Minute())) + case "second": + ls.Push(lua.LNumber(tm.Second())) + case "nanosecond": + ls.Push(lua.LNumber(tm.Nanosecond())) + default: + return 0 + } + return 1 +} + +func LuaBigIntIndex(ls *lua.LState) int { + _, bi := LuaBigInt.Check(ls, 1) + switch key := ls.Get(2).(type) { + case lua.LNumber: + ls.Push(lua.LNumber(bi.Bytes()[int(key)])) + case lua.LString: + switch string(key) { + case "sign": + ls.Push(lua.LNumber(bi.Sign())) + case "bytes": + ls.Push(lua.LString(bi.Bytes())) + } + default: + ls.RaiseError("BigInt accessed with non number/string") + } + return 1 +} + +func LuaBigIntLen(ls *lua.LState) int { + bi := LuaBigInt.StartMeta(ls) + ls.Push(lua.LNumber(len(bi.Bytes()))) + return 1 +} + +func LuaRatIndex(ls *lua.LState) int { + rat, key := LuaRat.StartIndex(ls) + switch key { + case "num": + ls.Push(LuaBigInt.New(ls, rat.Num())) + case "denom": + ls.Push(LuaBigInt.New(ls, rat.Denom())) + case "float64": + f64, _ := rat.Float64() + ls.Push(lua.LNumber(f64)) + default: + return 0 + } return 1 } diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index b2e5263937..02d5e8223f 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -251,8 +251,8 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { let snapshot_staging_path = match raw_options .remove("snapshot_staging_path") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), - _ => Some("".to_string()), + Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + _ => String::new(), }; let snapshot_max_parallel_workers: Option = match raw_options @@ -311,6 +311,11 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { _ => false, }; + let script = match raw_options.remove("script") { + Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + _ => String::new(), + }; + let flow_job = FlowJob { name: cdc.mirror_name.to_string().to_lowercase(), source_peer: cdc.source_peer.to_string().to_lowercase(), @@ -333,6 +338,7 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { soft_delete_col_name, synced_at_col_name, initial_snapshot_only: initial_copy_only, + script, }; if initial_copy_only && !do_initial_copy { diff --git a/nexus/catalog/migrations/V21__scripts.sql b/nexus/catalog/migrations/V23__scripts.sql similarity index 80% rename from nexus/catalog/migrations/V21__scripts.sql rename to nexus/catalog/migrations/V23__scripts.sql index 74060b25d6..d79c897510 100644 --- a/nexus/catalog/migrations/V21__scripts.sql +++ b/nexus/catalog/migrations/V23__scripts.sql @@ -1,6 +1,6 @@ CREATE TYPE script_lang AS ENUM ('lua'); -CREATE TABLE IF NOT EXISTS scripts ( +CREATE TABLE scripts ( id SERIAL PRIMARY KEY, lang script_lang NOT NULL, name TEXT NOT NULL UNIQUE, diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 4f6cace5f6..0d8b7ed6a6 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -325,12 +325,11 @@ impl Catalog { ) -> anyhow::Result { let peer_dbtype = self.get_peer_type_for_id(peer_id).await?; - let mut table_identifier_parts = table_identifier.split('.').collect::>(); - if table_identifier_parts.len() == 1 && (peer_dbtype != DbType::Bigquery) { - table_identifier_parts.insert(0, "public"); + if !table_identifier.contains('.') && peer_dbtype != DbType::Bigquery { + Ok(format!("public.{}", table_identifier)) + } else { + Ok(String::from(table_identifier)) } - - Ok(table_identifier_parts.join(".")) } pub async fn create_cdc_flow_job_entry(&self, job: &FlowJob) -> anyhow::Result<()> { diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index a962c4ec4f..b7569dbe45 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -161,7 +161,7 @@ impl FlowGrpcClient { snapshot_num_rows_per_partition: snapshot_num_rows_per_partition.unwrap_or(0), snapshot_max_parallel_workers: snapshot_max_parallel_workers.unwrap_or(0), snapshot_num_tables_in_parallel: snapshot_num_tables_in_parallel.unwrap_or(0), - snapshot_staging_path: job.snapshot_staging_path.clone().unwrap_or_default(), + snapshot_staging_path: job.snapshot_staging_path.clone(), cdc_staging_path: job.cdc_staging_path.clone().unwrap_or_default(), soft_delete: job.soft_delete, replication_slot_name: replication_slot_name.unwrap_or_default(), @@ -170,6 +170,7 @@ impl FlowGrpcClient { soft_delete_col_name: job.soft_delete_col_name.clone().unwrap_or_default(), synced_at_col_name: job.synced_at_col_name.clone().unwrap_or_default(), initial_snapshot_only: job.initial_snapshot_only, + script: job.script.clone(), ..Default::default() }; diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 021af81bbf..2946dd1d30 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -23,7 +23,7 @@ pub struct FlowJob { pub snapshot_num_rows_per_partition: Option, pub snapshot_max_parallel_workers: Option, pub snapshot_num_tables_in_parallel: Option, - pub snapshot_staging_path: Option, + pub snapshot_staging_path: String, pub cdc_staging_path: Option, pub soft_delete: bool, pub replication_slot_name: Option, @@ -34,6 +34,7 @@ pub struct FlowJob { pub soft_delete_col_name: Option, pub synced_at_col_name: Option, pub initial_snapshot_only: bool, + pub script: String, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/protos/flow.proto b/protos/flow.proto index f33b0696db..502520e1ea 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -69,6 +69,8 @@ message FlowConnectionConfigs { bool soft_delete = 17; string soft_delete_col_name = 18; string synced_at_col_name = 19; + + string script = 20; } message RenameTableOption { diff --git a/ui/app/api/peers/info/[peerName]/route.ts b/ui/app/api/peers/info/[peerName]/route.ts index 1e90bbdde6..61dc033efb 100644 --- a/ui/app/api/peers/info/[peerName]/route.ts +++ b/ui/app/api/peers/info/[peerName]/route.ts @@ -1,14 +1,16 @@ import prisma from '@/app/utils/prisma'; -import {NextRequest, NextResponse} from 'next/server'; +import { NextRequest, NextResponse } from 'next/server'; -import {getTruePeer} from '../../getTruePeer'; +import { getTruePeer } from '../../getTruePeer'; -export async function GET(request: NextRequest, - context: {params: {peerName: string}}) { +export async function GET( + request: NextRequest, + context: { params: { peerName: string } } +) { const peerName = context.params.peerName; const peer = await prisma.peers.findFirst({ - where : { - name : peerName, + where: { + name: peerName, }, }); const peerConfig = getTruePeer(peer!); diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index 0d70e3fac1..5d63cdd69a 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -139,4 +139,14 @@ export const cdcSettings: MirrorSetting[] = [ type: 'switch', advanced: true, }, + { + label: 'Script', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + script: (value as string) || '', + })), + tips: 'Associate PeerDB script with this mirror.', + advanced: true, + }, ]; diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index 34233f24fc..63dea81815 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -36,6 +36,7 @@ export const blankCDCSetting: FlowConnectionConfigs = { syncedAtColName: '', initialSnapshotOnly: false, idleTimeoutSeconds: 60, + script: '', }; export const blankQRepSetting = { diff --git a/ui/app/peers/create/[peerType]/helpers/ka.ts b/ui/app/peers/create/[peerType]/helpers/ka.ts index 8f542a9f9b..70bddbbe9f 100644 --- a/ui/app/peers/create/[peerType]/helpers/ka.ts +++ b/ui/app/peers/create/[peerType]/helpers/ka.ts @@ -42,6 +42,6 @@ export const blankKaSetting: KafkaConfig = { servers: [], username: '', password: '', - sasl: 'SCRAM-SHA-512', + sasl: 'PLAIN', disableTls: false, }; diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 440a718cdb..078d36b807 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -290,11 +290,18 @@ export const kaSchema = z.object({ .string({ required_error: 'Password is required' }) .min(1, { message: 'Password cannot be empty' }), sasl: z - .union([z.literal('SCRAM-SHA-256'), z.literal('SCRAM-SHA-512')], { - errorMap: () => ({ - message: 'Either SCRAM-SHA-256 or SCRAM-SHA-512 is required.', - }), - }) + .union( + [ + z.literal('PLAIN'), + z.literal('SCRAM-SHA-256'), + z.literal('SCRAM-SHA-512'), + ], + { + errorMap: () => ({ + message: 'One of PLAIN, SCRAM-SHA-256, or SCRAM-SHA-512 is required.', + }), + } + ) .optional(), disableTls: z.boolean().optional(), }); diff --git a/ui/components/PeerForms/KafkaConfig.tsx b/ui/components/PeerForms/KafkaConfig.tsx index e9acc40337..5b54078e7b 100644 --- a/ui/components/PeerForms/KafkaConfig.tsx +++ b/ui/components/PeerForms/KafkaConfig.tsx @@ -14,6 +14,7 @@ interface KafkaProps { } const saslOptions = [ + { value: 'PLAIN', label: 'PLAIN' }, { value: 'SCRAM-SHA-256', label: 'SCRAM-SHA-256' }, { value: 'SCRAM-SHA-512', label: 'SCRAM-SHA-512' }, ]; From a282e61b5ab26ca69c3072aba53ac0b72646cd2e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sat, 16 Mar 2024 19:40:59 +0000 Subject: [PATCH 04/31] replace big.Rat with decimal.Decimal, add more __tostring --- flow/pua/peerdb.go | 66 +++++++++++++++++++++++++++++++--------------- 1 file changed, 45 insertions(+), 21 deletions(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 8c1cefaf73..bc0b53fac6 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -11,6 +11,7 @@ import ( "github.com/google/uuid" "github.com/jackc/pgx/v5" + "github.com/shopspring/decimal" "github.com/yuin/gopher-lua" "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" @@ -19,15 +20,15 @@ import ( ) var ( - LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} - LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} - LuaQValue = LuaUserDataType[qvalue.QValue]{Name: "peerdb_value"} - LuaI64 = LuaUserDataType[int64]{Name: "flatbuffers_i64"} - LuaU64 = LuaUserDataType[uint64]{Name: "flatbuffers_u64"} - LuaTime = LuaUserDataType[time.Time]{Name: "peerdb_time"} - LuaUuid = LuaUserDataType[uuid.UUID]{Name: "peerdb_uuid"} - LuaBigInt = LuaUserDataType[*big.Int]{Name: "peerdb_bigint"} - LuaRat = LuaUserDataType[*big.Rat]{Name: "peerdb_bigrat"} + LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} + LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} + LuaQValue = LuaUserDataType[qvalue.QValue]{Name: "peerdb_value"} + LuaI64 = LuaUserDataType[int64]{Name: "flatbuffers_i64"} + LuaU64 = LuaUserDataType[uint64]{Name: "flatbuffers_u64"} + LuaTime = LuaUserDataType[time.Time]{Name: "peerdb_time"} + LuaUuid = LuaUserDataType[uuid.UUID]{Name: "peerdb_uuid"} + LuaBigInt = LuaUserDataType[*big.Int]{Name: "peerdb_bigint"} + LuaDecimal = LuaUserDataType[decimal.Decimal]{Name: "peerdb_bigrat"} ) func RegisterTypes(ls *lua.LState) { @@ -45,8 +46,8 @@ func RegisterTypes(ls *lua.LState) { mt = LuaQValue.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaQValueIndex)) - mt.RawSetString("__len", ls.NewFunction(LuaQValueLen)) mt.RawSetString("__tostring", ls.NewFunction(LuaQValueString)) + mt.RawSetString("__len", ls.NewFunction(LuaQValueLen)) mt = LuaUuid.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) @@ -68,13 +69,16 @@ func RegisterTypes(ls *lua.LState) { mt = LuaTime.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaTimeIndex)) + mt.RawSetString("__tostring", ls.NewFunction(LuaTimeString)) mt = LuaBigInt.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaBigIntIndex)) + mt.RawSetString("__tostring", ls.NewFunction(LuaBigIntString)) mt.RawSetString("__len", ls.NewFunction(LuaBigIntLen)) - mt = LuaRat.NewMetatable(ls) - mt.RawSetString("__index", ls.NewFunction(LuaRatIndex)) + mt = LuaDecimal.NewMetatable(ls) + mt.RawSetString("__index", ls.NewFunction(LuaDecimalIndex)) + mt.RawSetString("__tostring", ls.NewFunction(LuaDecimalString)) peerdb := ls.NewTable() peerdb.RawSetString("RowToJSON", ls.NewFunction(LuaRowToJSON)) @@ -259,8 +263,8 @@ func LuaQValueIndex(ls *lua.LState) int { } case time.Time: ls.Push(LuaTime.New(ls, v)) - case *big.Rat: - ls.Push(LuaRat.New(ls, v)) + case decimal.Decimal: + ls.Push(LuaDecimal.New(ls, v)) case [16]byte: ls.Push(LuaUuid.New(ls, uuid.UUID(v))) case []byte: @@ -540,6 +544,12 @@ func LuaTimeIndex(ls *lua.LState) int { return 1 } +func LuaTimeString(ls *lua.LState) int { + tm := LuaTime.StartMeta(ls) + ls.Push(lua.LString(tm.String())) + return 1 +} + func LuaBigIntIndex(ls *lua.LState) int { _, bi := LuaBigInt.Check(ls, 1) switch key := ls.Get(2).(type) { @@ -558,24 +568,38 @@ func LuaBigIntIndex(ls *lua.LState) int { return 1 } +func LuaBigIntString(ls *lua.LState) int { + bi := LuaBigInt.StartMeta(ls) + ls.Push(lua.LString(bi.String())) + return 1 +} + func LuaBigIntLen(ls *lua.LState) int { bi := LuaBigInt.StartMeta(ls) ls.Push(lua.LNumber(len(bi.Bytes()))) return 1 } -func LuaRatIndex(ls *lua.LState) int { - rat, key := LuaRat.StartIndex(ls) +func LuaDecimalIndex(ls *lua.LState) int { + num, key := LuaDecimal.StartIndex(ls) switch key { - case "num": - ls.Push(LuaBigInt.New(ls, rat.Num())) - case "denom": - ls.Push(LuaBigInt.New(ls, rat.Denom())) + case "exponent": + ls.Push(lua.LNumber(num.Exponent())) + case "bigint": + ls.Push(LuaBigInt.New(ls, num.BigInt())) + case "int64": + ls.Push(LuaI64.New(ls, num.IntPart())) case "float64": - f64, _ := rat.Float64() + f64, _ := num.Float64() ls.Push(lua.LNumber(f64)) default: return 0 } return 1 } + +func LuaDecimalString(ls *lua.LState) int { + num := LuaDecimal.StartMeta(ls) + ls.Push(lua.LString(num.String())) + return 1 +} From 1c6376a4e56be6ce4ec93f17c67a64f86ce17a1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sun, 17 Mar 2024 11:35:44 +0000 Subject: [PATCH 05/31] Fix CreateByteVector, expose more decimal/int properties --- flow/pua/flatbuffers_builder.go | 20 ++++++++++++++++---- flow/pua/peerdb.go | 8 ++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers_builder.go index 0c024446c9..7fd585f486 100644 --- a/flow/pua/flatbuffers_builder.go +++ b/flow/pua/flatbuffers_builder.go @@ -227,7 +227,7 @@ func FlatBuffers_Builder_Loader(ls *lua.LState) int { "StartVector": BuilderStartVector, "EndVector": BuilderEndVector, "CreateString": BuilderCreateString, - "CreateByteVector": BuilderCreateString, + "CreateByteVector": BuilderCreateByteVector, "Slot": BuilderSlot, "Finish": BuilderFinish, "FinishSizePrefixed": BuilderFinishSizePrefixed, @@ -407,7 +407,7 @@ func BuilderEndVector(ls *lua.LState) int { return 1 } -func BuilderCreateString(ls *lua.LState) int { +func createBytesHelper(ls *lua.LState, addnul bool) int { b := LuaBuilder.StartMeta(ls) s := ls.CheckString(2) if b.nested { @@ -417,8 +417,12 @@ func BuilderCreateString(ls *lua.LState) int { b.nested = true lens := len(s) - b.Prep(4, lens+1) - b.PlaceU64(0, uint8n) + if addnul { + b.Prep(4, lens+1) + b.PlaceU64(0, uint8n) + } else { + b.Prep(4, lens) + } b.head -= lens copy(b.ba.data[b.head:], s) @@ -426,6 +430,14 @@ func BuilderCreateString(ls *lua.LState) int { return 1 } +func BuilderCreateString(ls *lua.LState) int { + return createBytesHelper(ls, true) +} + +func BuilderCreateByteVector(ls *lua.LState) int { + return createBytesHelper(ls, false) +} + func BuilderSlot(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) slotnum := int(ls.CheckNumber(2)) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index bc0b53fac6..ece6daa9b3 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -561,6 +561,10 @@ func LuaBigIntIndex(ls *lua.LState) int { ls.Push(lua.LNumber(bi.Sign())) case "bytes": ls.Push(lua.LString(bi.Bytes())) + case "int64": + ls.Push(LuaI64.New(ls, bi.Int64())) + case "is64": + ls.Push(lua.LBool(bi.IsInt64())) } default: ls.RaiseError("BigInt accessed with non number/string") @@ -583,6 +587,10 @@ func LuaBigIntLen(ls *lua.LState) int { func LuaDecimalIndex(ls *lua.LState) int { num, key := LuaDecimal.StartIndex(ls) switch key { + case "coefficient": + ls.Push(LuaBigInt.New(ls, num.Coefficient())) + case "coefficient64": + ls.Push(LuaI64.New(ls, num.CoefficientInt64())) case "exponent": ls.Push(lua.LNumber(num.Exponent())) case "bigint": From f3c957b588b0410a106494781e63fcb08e7df80d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sun, 17 Mar 2024 12:17:12 +0000 Subject: [PATCH 06/31] add peerdb.type function to help with debugging --- flow/pua/peerdb.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index ece6daa9b3..86defca9fa 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -85,6 +85,7 @@ func RegisterTypes(ls *lua.LState) { peerdb.RawSetString("RowColumns", ls.NewFunction(LuaRowColumns)) peerdb.RawSetString("Now", ls.NewFunction(LuaNow)) peerdb.RawSetString("UUID", ls.NewFunction(LuaUUID)) + peerdb.RawSetString("type", ls.NewFunction(LuaType)) ls.Env.RawSetString("peerdb", peerdb) } @@ -359,6 +360,15 @@ func LuaUUID(ls *lua.LState) int { return 1 } +func LuaType(ls *lua.LState) int { + val := ls.Get(1) + if ud, ok := val.(*lua.LUserData); ok { + ls.Push(lua.LString(fmt.Sprintf("%T", ud.Value))) + return 1 + } + return 0 +} + func Lua64Eq(ls *lua.LState) int { aud := ls.CheckUserData(1) bud := ls.CheckUserData(2) From 3e902717e2cc0604229f8a25561375d21dacf825 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sun, 17 Mar 2024 12:51:27 +0000 Subject: [PATCH 07/31] Don't expose QValue directly to scripts 2 reasons: 1. I'd like to eventually replace QValue with interface{} 2. Avoids having to sprinkle `.value` throughout script --- flow/pua/peerdb.go | 215 ++++++++++++++++++++------------------------- 1 file changed, 96 insertions(+), 119 deletions(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 86defca9fa..bbcf9cc508 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -4,9 +4,7 @@ import ( "bytes" "fmt" "math/big" - "reflect" "strconv" - "strings" "time" "github.com/google/uuid" @@ -22,7 +20,6 @@ import ( var ( LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} - LuaQValue = LuaUserDataType[qvalue.QValue]{Name: "peerdb_value"} LuaI64 = LuaUserDataType[int64]{Name: "flatbuffers_i64"} LuaU64 = LuaUserDataType[uint64]{Name: "flatbuffers_u64"} LuaTime = LuaUserDataType[time.Time]{Name: "peerdb_time"} @@ -44,11 +41,6 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__index", ls.NewFunction(LuaRowIndex)) mt.RawSetString("__len", ls.NewFunction(LuaRowLen)) - mt = LuaQValue.NewMetatable(ls) - mt.RawSetString("__index", ls.NewFunction(LuaQValueIndex)) - mt.RawSetString("__tostring", ls.NewFunction(LuaQValueString)) - mt.RawSetString("__len", ls.NewFunction(LuaQValueLen)) - mt = LuaUuid.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaUuidString)) @@ -83,9 +75,11 @@ func RegisterTypes(ls *lua.LState) { peerdb := ls.NewTable() peerdb.RawSetString("RowToJSON", ls.NewFunction(LuaRowToJSON)) peerdb.RawSetString("RowColumns", ls.NewFunction(LuaRowColumns)) + peerdb.RawSetString("RowColumnKind", ls.NewFunction(LuaRowColumnKind)) peerdb.RawSetString("Now", ls.NewFunction(LuaNow)) peerdb.RawSetString("UUID", ls.NewFunction(LuaUUID)) peerdb.RawSetString("type", ls.NewFunction(LuaType)) + peerdb.RawSetString("tostring", ls.NewFunction(LuaToString)) ls.Env.RawSetString("peerdb", peerdb) } @@ -117,16 +111,18 @@ func LoadPeerdbScript(ls *lua.LState) int { return 1 } -func LuaRowIndex(ls *lua.LState) int { - row, key := LuaRow.StartIndex(ls) - - qv, err := row.GetValueByColName(key) +func GetRowQ(ls *lua.LState, row *model.RecordItems, col string) qvalue.QValue { + qv, err := row.GetValueByColName(col) if err != nil { ls.RaiseError(err.Error()) - return 0 + return qvalue.QValue{} } + return qv +} - ls.Push(LuaQValue.New(ls, qv)) +func LuaRowIndex(ls *lua.LState) int { + row, key := LuaRow.StartIndex(ls) + ls.Push(LuaQValue(ls, GetRowQ(ls, row, key))) return 1 } @@ -157,6 +153,12 @@ func LuaRowColumns(ls *lua.LState) int { return 1 } +func LuaRowColumnKind(ls *lua.LState) int { + row, key := LuaRow.StartIndex(ls) + ls.Push(lua.LString(GetRowQ(ls, row, key).Kind)) + return 1 +} + func LuaRecordIndex(ls *lua.LState) int { record, key := LuaRecord.StartIndex(ls) switch key { @@ -216,121 +218,87 @@ func LuaRecordIndex(ls *lua.LState) int { return 1 } -func qvToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) { +func qvToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) *lua.LTable { tbl := ls.CreateTable(len(s), 0) for idx, val := range s { tbl.RawSetInt(idx, f(val)) } - ls.Push(tbl) + return tbl } -func LuaQValueIndex(ls *lua.LState) int { - qv, key := LuaQValue.StartIndex(ls) - switch key { - case "kind": - ls.Push(lua.LString(qv.Kind)) - case "value": - switch v := qv.Value.(type) { - case nil: - ls.Push(lua.LNil) - case bool: - ls.Push(lua.LBool(v)) - case uint8: - if qv.Kind == qvalue.QValueKindQChar { - ls.Push(lua.LString(rune(v))) - } else { - ls.Push(lua.LNumber(v)) - } - case int16: - ls.Push(lua.LNumber(v)) - case int32: - ls.Push(lua.LNumber(v)) - case int64: - ls.Push(LuaI64.New(ls, v)) - case float32: - ls.Push(lua.LNumber(v)) - case float64: - ls.Push(lua.LNumber(v)) - case string: - if qv.Kind == qvalue.QValueKindUUID { - u, err := uuid.Parse(v) - if err != nil { - ls.Push(LuaUuid.New(ls, u)) - } else { - ls.Push(lua.LString(v)) - } - } else { - ls.Push(lua.LString(v)) +func LuaQValue(ls *lua.LState, qv qvalue.QValue) lua.LValue { + switch v := qv.Value.(type) { + case nil: + return lua.LNil + case bool: + return lua.LBool(v) + case uint8: + if qv.Kind == qvalue.QValueKindQChar { + return lua.LString(rune(v)) + } else { + return lua.LNumber(v) + } + case int16: + return lua.LNumber(v) + case int32: + return lua.LNumber(v) + case int64: + return LuaI64.New(ls, v) + case float32: + return lua.LNumber(v) + case float64: + return lua.LNumber(v) + case string: + if qv.Kind == qvalue.QValueKindUUID { + u, err := uuid.Parse(v) + if err != nil { + return LuaUuid.New(ls, u) } - case time.Time: - ls.Push(LuaTime.New(ls, v)) - case decimal.Decimal: - ls.Push(LuaDecimal.New(ls, v)) - case [16]byte: - ls.Push(LuaUuid.New(ls, uuid.UUID(v))) - case []byte: - ls.Push(lua.LString(v)) - case []float32: - qvToLTable(ls, v, func(f float32) lua.LValue { - return lua.LNumber(f) - }) - case []float64: - qvToLTable(ls, v, func(f float64) lua.LValue { - return lua.LNumber(f) - }) - case []int16: - qvToLTable(ls, v, func(f int16) lua.LValue { - return lua.LNumber(f) - }) - case []int32: - qvToLTable(ls, v, func(f int32) lua.LValue { - return lua.LNumber(f) - }) - case []int64: - qvToLTable(ls, v, func(x int64) lua.LValue { - return LuaI64.New(ls, x) - }) - case []string: - qvToLTable(ls, v, func(x string) lua.LValue { - return lua.LString(x) - }) - case []time.Time: - qvToLTable(ls, v, func(x time.Time) lua.LValue { - return LuaTime.New(ls, x) - }) - case []bool: - qvToLTable(ls, v, func(x bool) lua.LValue { - return lua.LBool(x) - }) } - case "int64": - ls.Push(LuaI64.New(ls, reflect.ValueOf(qv.Value).Int())) - case "float64": - ls.Push(lua.LNumber(reflect.ValueOf(qv.Value).Float())) + return lua.LString(v) + case time.Time: + return LuaTime.New(ls, v) + case decimal.Decimal: + return LuaDecimal.New(ls, v) + case [16]byte: + return LuaUuid.New(ls, uuid.UUID(v)) + case []byte: + return lua.LString(v) + case []float32: + return qvToLTable(ls, v, func(f float32) lua.LValue { + return lua.LNumber(f) + }) + case []float64: + return qvToLTable(ls, v, func(f float64) lua.LValue { + return lua.LNumber(f) + }) + case []int16: + return qvToLTable(ls, v, func(x int16) lua.LValue { + return lua.LNumber(x) + }) + case []int32: + return qvToLTable(ls, v, func(x int32) lua.LValue { + return lua.LNumber(x) + }) + case []int64: + return qvToLTable(ls, v, func(x int64) lua.LValue { + return LuaI64.New(ls, x) + }) + case []string: + return qvToLTable(ls, v, func(x string) lua.LValue { + return lua.LString(x) + }) + case []time.Time: + return qvToLTable(ls, v, func(x time.Time) lua.LValue { + return LuaTime.New(ls, x) + }) + case []bool: + return qvToLTable(ls, v, func(x bool) lua.LValue { + return lua.LBool(x) + }) default: - return 0 - } - return 1 -} - -func LuaQValueLen(ls *lua.LState) int { - qv := LuaQValue.StartMeta(ls) - str, ok := qv.Value.(string) - if ok { - ls.Push(lua.LNumber(len(str))) - return 1 - } - if strings.HasPrefix(string(qv.Kind), "array_") { - ls.Push(lua.LNumber(reflect.ValueOf(qv.Value).Len())) - return 1 + return lua.LString(fmt.Sprint(qv.Value)) } - return 0 -} - -func LuaQValueString(ls *lua.LState) int { - qv := LuaQValue.StartMeta(ls) - ls.Push(lua.LString(fmt.Sprint(qv.Value))) - return 1 } func LuaUuidIndex(ls *lua.LState) int { @@ -369,6 +337,15 @@ func LuaType(ls *lua.LState) int { return 0 } +func LuaToString(ls *lua.LState) int { + val := ls.Get(1) + if ud, ok := val.(*lua.LUserData); ok { + ls.Push(lua.LString(fmt.Sprint(ud.Value))) + return 1 + } + return 0 +} + func Lua64Eq(ls *lua.LState) int { aud := ls.CheckUserData(1) bud := ls.CheckUserData(2) From 6107a89c46af05910be358bb17a527bcbd6a16db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 18 Mar 2024 02:49:02 +0000 Subject: [PATCH 08/31] 3 tweaks: LState: IncludeGoStackTraces peerdb.RowColumns: start indexing at 1 Time: rename `unix` to `unix_seconds`, expose `unix` as returning a floating point number of seconds since unix epoch --- flow/connectors/kafka/kafka.go | 2 +- flow/pua/peerdb.go | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 920d14c872..f007922f46 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -137,7 +137,7 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords var fn *lua.LFunction var ls *lua.LState if req.Script != "" { - ls = lua.NewState(lua.Options{SkipOpenLibs: true}) + ls = lua.NewState(lua.Options{SkipOpenLibs: true, IncludeGoStackTrace: true}) defer ls.Close() ls.SetContext(wgCtx) for _, pair := range []struct { diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index bbcf9cc508..44dade8a89 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -147,7 +147,7 @@ func LuaRowColumns(ls *lua.LState) int { _, row := LuaRow.Check(ls, 1) tbl := ls.CreateTable(len(row.ColToValIdx), 0) for col, idx := range row.ColToValIdx { - tbl.RawSetInt(idx, lua.LString(col)) + tbl.RawSetInt(idx+1, lua.LString(col)) } ls.Push(tbl) return 1 @@ -507,8 +507,10 @@ func LuaTimeIndex(ls *lua.LState) int { ls.Push(LuaI64.New(ls, tm.UnixMicro())) case "unix_milli": ls.Push(LuaI64.New(ls, tm.UnixMilli())) - case "unix": + case "unix_second": ls.Push(LuaI64.New(ls, tm.Unix())) + case "unix": + ls.Push(lua.LNumber(float64(tm.Unix()) + float64(tm.Nanosecond())/1e9)) case "year": ls.Push(lua.LNumber(tm.Year())) case "month": From 9ef43f24bee8a0741c10881059263c2185129669 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 18 Mar 2024 15:41:51 +0000 Subject: [PATCH 09/31] fix resize not updating data --- flow/pua/flatbuffers_builder.go | 1 + 1 file changed, 1 insertion(+) diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers_builder.go index 7fd585f486..86bcefbe69 100644 --- a/flow/pua/flatbuffers_builder.go +++ b/flow/pua/flatbuffers_builder.go @@ -65,6 +65,7 @@ func (b *Builder) Prep(width uint8, additional int) { newdata = newdata[:cap(newdata)] copy(newdata[:oldlen], newdata[len(newdata)-oldlen:]) b.head += len(newdata) - oldlen + b.ba.data = newdata } b.Pad(alignsize) From 2d66ed14fddecd58866318e29a46d2c0be3a92a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 18 Mar 2024 17:59:20 +0000 Subject: [PATCH 10/31] Fix flatbuffers not handling buffer resizing properly --- flow/pua/flatbuffers_builder.go | 55 ++++++++++++++------------------- 1 file changed, 24 insertions(+), 31 deletions(-) diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers_builder.go index 86bcefbe69..4cde881345 100644 --- a/flow/pua/flatbuffers_builder.go +++ b/flow/pua/flatbuffers_builder.go @@ -54,7 +54,7 @@ func (b *Builder) Prep(width uint8, additional int) { if width > b.minalign { b.minalign = width } - k := len(b.ba.data) - b.head + additional + k := b.Offset() + additional alignsize := -k & int(width-1) space := alignsize + int(width) + additional @@ -63,7 +63,7 @@ func (b *Builder) Prep(width uint8, additional int) { oldlen := len(b.ba.data) newdata := slices.Grow(b.ba.data, space) newdata = newdata[:cap(newdata)] - copy(newdata[:oldlen], newdata[len(newdata)-oldlen:]) + copy(newdata[len(newdata)-oldlen:], newdata[:oldlen]) b.head += len(newdata) - oldlen b.ba.data = newdata } @@ -133,8 +133,8 @@ func (b *Builder) Slot(ls *lua.LState, slotnum int) { ls.RaiseError("Slot called outside nested context") return } - if slotnum >= len(b.currentVT) { - b.currentVT = slices.Grow(b.currentVT, slotnum-len(b.currentVT)+1) + for slotnum >= len(b.currentVT) { + b.currentVT = append(b.currentVT, 0) } b.currentVT[slotnum] = b.Offset() } @@ -145,7 +145,7 @@ func vtableEqual(a []int, objectStart int, b []byte) bool { } for i, ai := range a { - x := int16n.UnpackU64(b[i*2:]) + x := uint16n.UnpackU64(b[i*2:]) if (x != 0 || ai != 0) && int(x) != objectStart-ai { return false } @@ -157,8 +157,8 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { b.PrependSOffsetTRelative(ls, 0) objectOffset := b.Offset() - for len(b.ba.data) > 0 && b.ba.data[len(b.ba.data)-1] == 0 { - b.ba.data = b.ba.data[:len(b.ba.data)-1] + for len(b.currentVT) > 0 && b.currentVT[len(b.currentVT)-1] == 0 { + b.currentVT = b.currentVT[:len(b.currentVT)-1] } var existingVtable int @@ -166,11 +166,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { vt2Offset := b.vtables[i] vt2Start := len(b.ba.data) - vt2Offset vt2Len := uint16n.UnpackU64(b.ba.data[vt2Start:]) - - metadata := VtableMetadataFields * 2 - vt2End := vt2Start + int(vt2Len) - vt2 := b.ba.data[vt2Start+metadata : vt2End] - + vt2 := b.ba.data[vt2Start+VtableMetadataFields*2 : vt2Start+int(vt2Len)] if vtableEqual(b.currentVT, objectOffset, vt2) { existingVtable = vt2Offset break @@ -186,20 +182,16 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { b.PrependVOffsetT(off) } - objectSize := uint16(objectOffset - b.objectEnd) - b.PrependVOffsetT(objectSize) - - vBytes := uint16(len(b.currentVT)+VtableMetadataFields) * 2 - b.PrependVOffsetT(vBytes) + // end each vtable with object size & vtable size + b.PrependVOffsetT(uint16(objectOffset - b.objectEnd)) + b.PrependVOffsetT(uint16(len(b.currentVT)+VtableMetadataFields) * 2) - objectStart := len(b.ba.data) - objectOffset newOffset := b.Offset() - int32n.PackU64(b.ba.data[newOffset-objectOffset:], uint64(objectStart)) - + int32n.PackU64(b.ba.data[len(b.ba.data)-objectOffset:], uint64(newOffset-objectOffset)) b.vtables = append(b.vtables, newOffset) } else { b.head = len(b.ba.data) - objectOffset - int16n.PackU64(b.ba.data[b.head:], uint64(existingVtable-objectOffset)) + int32n.PackU64(b.ba.data[b.head:], uint64(existingVtable-objectOffset)) } if len(b.currentVT) != 0 { @@ -278,8 +270,8 @@ func BuilderNew(ls *lua.LState) int { ls.Push(LuaBuilder.New(ls, &Builder{ ba: BinaryArray{data: make([]byte, initialSize)}, - vtables: nil, - currentVT: nil, + vtables: make([]int, 0, 4), + currentVT: make([]int, 0, 4), head: initialSize, objectEnd: 0, finished: false, @@ -297,9 +289,7 @@ func BuilderClear(ls *lua.LState) int { if len(b.vtables) != 0 { b.vtables = b.vtables[:0] } - if len(b.currentVT) != 0 { - b.currentVT = b.currentVT[:0] - } + b.currentVT = b.currentVT[:0] b.objectEnd = 0 b.head = len(b.ba.data) return 0 @@ -324,7 +314,7 @@ func BuilderStartObject(ls *lua.LState) int { b.nested = true numFields := int(ls.CheckNumber(2)) - b.currentVT = make([]int, numFields) + b.currentVT = slices.Grow(b.currentVT[:0], numFields)[:0] b.objectEnd = b.Offset() return 0 } @@ -354,7 +344,7 @@ func BuilderHead(ls *lua.LState) int { func BuilderOffset(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) - ls.Push(lua.LNumber(len(b.ba.data) - b.head)) + ls.Push(lua.LNumber(b.Offset())) return 1 } @@ -412,7 +402,11 @@ func createBytesHelper(ls *lua.LState, addnul bool) int { b := LuaBuilder.StartMeta(ls) s := ls.CheckString(2) if b.nested { - ls.RaiseError("CreateString called in nested context") + if addnul { + ls.RaiseError("CreateString called in nested context") + } else { + ls.RaiseError("CreateByteVector called in nested context") + } return 0 } b.nested = true @@ -458,8 +452,7 @@ func FinishHelper(ls *lua.LState, sizePrefix bool) int { b.Prep(b.minalign, additional) b.PrependUOffsetTRelative(ls, rootTable) if sizePrefix { - size := len(b.ba.data) - b.head - b.PrependU64(int32n, uint64(size)) + b.PrependU64(int32n, uint64(b.Offset())) } b.finished = true ls.Push(lua.LNumber(b.head)) From 8f97c6800cce9f5d0a3bffe1b22f3506fd13e9eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 18 Mar 2024 20:12:22 +0000 Subject: [PATCH 11/31] Kafka: username optional, if not supplied no auth used --- ui/app/peers/create/[peerType]/helpers/ch.ts | 1 + ui/app/peers/create/[peerType]/helpers/ka.ts | 3 +++ ui/app/peers/create/[peerType]/schema.ts | 12 +++++------- ui/components/PeerForms/KafkaConfig.tsx | 1 - 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/ui/app/peers/create/[peerType]/helpers/ch.ts b/ui/app/peers/create/[peerType]/helpers/ch.ts index bda5b8eb1e..29147e0620 100644 --- a/ui/app/peers/create/[peerType]/helpers/ch.ts +++ b/ui/app/peers/create/[peerType]/helpers/ch.ts @@ -45,6 +45,7 @@ export const clickhouseSetting: PeerSetting[] = [ setter((curr) => ({ ...curr, disableTls: value as boolean })), type: 'switch', tips: 'If you are using a non-TLS connection for Clickhouse server, check this box.', + optional: true, }, { label: 'S3 Path', diff --git a/ui/app/peers/create/[peerType]/helpers/ka.ts b/ui/app/peers/create/[peerType]/helpers/ka.ts index 70bddbbe9f..4617e7360b 100644 --- a/ui/app/peers/create/[peerType]/helpers/ka.ts +++ b/ui/app/peers/create/[peerType]/helpers/ka.ts @@ -14,12 +14,14 @@ export const kaSetting: PeerSetting[] = [ label: 'Username', stateHandler: (value, setter) => setter((curr) => ({ ...curr, username: value as string })), + optional: true, }, { label: 'Password', type: 'password', stateHandler: (value, setter) => setter((curr) => ({ ...curr, password: value as string })), + optional: true, }, { label: 'SASL Mechanism', @@ -35,6 +37,7 @@ export const kaSetting: PeerSetting[] = [ setter((curr) => ({ ...curr, disableTls: value as boolean })), type: 'switch', tips: 'If you are using a non-TLS connection for Kafka server, check this box.', + optional: true, }, ]; diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 078d36b807..f9541ac274 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -282,13 +282,11 @@ export const chSchema = z.object({ }); export const kaSchema = z.object({ - servers: z.array(z.string()).min(1, { message: 'Atleast 1 server required' }), - username: z - .string({ required_error: 'Username is required' }) - .min(1, { message: 'Username cannot be empty' }), - password: z - .string({ required_error: 'Password is required' }) - .min(1, { message: 'Password cannot be empty' }), + servers: z + .array(z.string({ required_error: 'Server address must not be empty' })) + .min(1, { message: 'At least 1 server required' }), + username: z.string().optional(), + password: z.string().optional(), sasl: z .union( [ diff --git a/ui/components/PeerForms/KafkaConfig.tsx b/ui/components/PeerForms/KafkaConfig.tsx index 5b54078e7b..0e0549429a 100644 --- a/ui/components/PeerForms/KafkaConfig.tsx +++ b/ui/components/PeerForms/KafkaConfig.tsx @@ -22,7 +22,6 @@ const saslOptions = [ const KafkaForm = ({ setter }: KafkaProps) => { return (
- {kaSetting.map((setting, index) => { return setting.type === 'switch' ? ( Date: Mon, 18 Mar 2024 22:42:00 +0000 Subject: [PATCH 12/31] Move flatbuffers into its own package, fix stack not being cleared in kafka code --- flow/connectors/kafka/kafka.go | 12 +-- flow/pua/{ => flatbuffers}/flatbuffers.go | 4 +- .../flatbuffers_binaryarray.go | 49 +++++------ .../{ => flatbuffers}/flatbuffers_builder.go | 44 +++++----- .../{ => flatbuffers}/flatbuffers_numtypes.go | 85 +++++++++---------- .../pua/{ => flatbuffers}/flatbuffers_view.go | 48 +++++------ 6 files changed, 116 insertions(+), 126 deletions(-) rename flow/pua/{ => flatbuffers}/flatbuffers.go (93%) rename flow/pua/{ => flatbuffers}/flatbuffers_binaryarray.go (74%) rename flow/pua/{ => flatbuffers}/flatbuffers_builder.go (94%) rename flow/pua/{ => flatbuffers}/flatbuffers_numtypes.go (74%) rename flow/pua/{ => flatbuffers}/flatbuffers_view.go (76%) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index f007922f46..5bcfe839a6 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -22,6 +22,7 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/pua" + "github.com/PeerDB-io/peer-flow/pua/flatbuffers" ) type KafkaConnector struct { @@ -137,7 +138,7 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords var fn *lua.LFunction var ls *lua.LState if req.Script != "" { - ls = lua.NewState(lua.Options{SkipOpenLibs: true, IncludeGoStackTrace: true}) + ls = lua.NewState(lua.Options{SkipOpenLibs: true}) defer ls.Close() ls.SetContext(wgCtx) for _, pair := range []struct { @@ -157,7 +158,7 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("failed to initialize Lua runtime: %w", err) } } - ls.PreloadModule("flatbuffers", pua.FlatBuffers_Loader) + ls.PreloadModule("flatbuffers", pua_flatbuffers.Loader) pua.RegisterTypes(ls) ls.Env.RawSetString("print", ls.NewFunction(func(ls *lua.LState) int { top := ls.GetTop() @@ -191,7 +192,6 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords if err := wgCtx.Err(); err != nil { return nil, err } - topic := record.GetDestinationTableName() ls.Push(fn) ls.Push(pua.LuaRecord.New(ls, record)) err := ls.PCall(1, 1, nil) @@ -199,14 +199,16 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("script failed: %w", err) } value := ls.Get(-1) + ls.SetTop(0) if value != lua.LNil { lstr, ok := value.(lua.LString) if !ok { - return nil, fmt.Errorf("script returned non-nil non-string: %v", value) + return nil, fmt.Errorf("script returned non-nil non-string: %s", value) } + wg.Add(1) + topic := record.GetDestinationTableName() c.client.Produce(wgCtx, &kgo.Record{Topic: topic, Value: []byte(lstr)}, produceCb) - numRecords += 1 tableNameRowsMapping[topic] += 1 } diff --git a/flow/pua/flatbuffers.go b/flow/pua/flatbuffers/flatbuffers.go similarity index 93% rename from flow/pua/flatbuffers.go rename to flow/pua/flatbuffers/flatbuffers.go index c27431007a..2163417d30 100644 --- a/flow/pua/flatbuffers.go +++ b/flow/pua/flatbuffers/flatbuffers.go @@ -1,10 +1,10 @@ -package pua +package pua_flatbuffers import ( "github.com/yuin/gopher-lua" ) -func FlatBuffers_Loader(ls *lua.LState) int { +func Loader(ls *lua.LState) int { ls.PreloadModule("flatbuffers.binaryarray", FlatBuffers_BinaryArray_Loader) ls.PreloadModule("flatbuffers.builder", FlatBuffers_Builder_Loader) ls.PreloadModule("flatbuffers.numTypes", FlatBuffers_N_Loader) diff --git a/flow/pua/flatbuffers_binaryarray.go b/flow/pua/flatbuffers/flatbuffers_binaryarray.go similarity index 74% rename from flow/pua/flatbuffers_binaryarray.go rename to flow/pua/flatbuffers/flatbuffers_binaryarray.go index 635e8f7f55..caa087c7fc 100644 --- a/flow/pua/flatbuffers_binaryarray.go +++ b/flow/pua/flatbuffers/flatbuffers_binaryarray.go @@ -1,14 +1,12 @@ -package pua +package pua_flatbuffers import ( "github.com/yuin/gopher-lua" -) -type BinaryArray struct { - data []byte -} + "github.com/PeerDB-io/peer-flow/pua" +) -var LuaBinaryArray = LuaUserDataType[BinaryArray]{Name: "flatbuffers_binaryarray"} +var LuaBinaryArray = pua.LuaUserDataType[[]byte]{Name: "flatbuffers_binaryarray"} func FlatBuffers_BinaryArray_Loader(ls *lua.LState) int { m := ls.NewTable() @@ -28,16 +26,12 @@ func FlatBuffers_BinaryArray_Loader(ls *lua.LState) int { func BinaryArrayNew(ls *lua.LState) int { lval := ls.Get(1) - var ba BinaryArray + var ba []byte switch val := lval.(type) { case lua.LString: - ba = BinaryArray{ - data: []byte(val), - } + ba = []byte(val) case lua.LNumber: - ba = BinaryArray{ - data: make([]byte, int(val)), - } + ba = make([]byte, int(val)) default: ls.RaiseError("Expect a integer size value or string to construct a binary array") return 0 @@ -48,7 +42,7 @@ func BinaryArrayNew(ls *lua.LState) int { func BinaryArrayLen(ls *lua.LState) int { ba := LuaBinaryArray.StartMeta(ls) - ls.Push(lua.LNumber(len(ba.data))) + ls.Push(lua.LNumber(len(ba))) return 1 } @@ -56,9 +50,9 @@ func BinaryArrayIndex(ls *lua.LState) int { ba, key := LuaBinaryArray.StartIndex(ls) switch key { case "size": - ls.Push(lua.LNumber(len(ba.data))) + ls.Push(lua.LNumber(len(ba))) case "str": - ls.Push(lua.LString(ba.data)) + ls.Push(lua.LString(ba)) case "data": ls.RaiseError("BinaryArray data property inaccessible") return 0 @@ -77,29 +71,28 @@ func BinaryArraySlice(ls *lua.LState) int { startPos = 0 } if luaEndPos, ok := ls.Get(3).(lua.LNumber); ok { - endPos = min(int(luaEndPos), len(ba.data)) + endPos = min(int(luaEndPos), len(ba)) } else { - endPos = len(ba.data) + endPos = len(ba) } - ls.Push(lua.LString(ba.data[startPos:endPos])) + ls.Push(lua.LString(ba[startPos:endPos])) return 1 } func BinaryArrayGrow(ls *lua.LState) int { baud, ba := LuaBinaryArray.Check(ls, 1) newsize := int(ls.CheckNumber(2)) - if newsize > len(ba.data) { + if newsize > len(ba) { newdata := make([]byte, newsize) - copy(newdata[newsize-len(ba.data):], ba.data) - ba.data = newdata - baud.Value = ba + copy(newdata[newsize-len(ba):], ba) + baud.Value = newdata } return 0 } -func (ba *BinaryArray) Pad(n int, start int) { +func Pad(ba []byte, n int, start int) { for i := range n { - ba.data[start+i] = 0 + ba[start+i] = 0 } } @@ -107,7 +100,7 @@ func BinaryArrayPad(ls *lua.LState) int { ba := LuaBinaryArray.StartMeta(ls) n := int(ls.CheckNumber(2)) startPos := int(ls.CheckNumber(3)) - ba.Pad(n, startPos) + Pad(ba, n, startPos) return 0 } @@ -116,10 +109,10 @@ func BinaryArraySet(ls *lua.LState) int { idx := int(ls.CheckNumber(3)) value := ls.Get(2) if num, ok := value.(lua.LNumber); ok { - ba.data[idx] = byte(num) + ba[idx] = byte(num) } if str, ok := value.(lua.LString); ok { - ba.data[idx] = str[0] + ba[idx] = str[0] } return 0 } diff --git a/flow/pua/flatbuffers_builder.go b/flow/pua/flatbuffers/flatbuffers_builder.go similarity index 94% rename from flow/pua/flatbuffers_builder.go rename to flow/pua/flatbuffers/flatbuffers_builder.go index 4cde881345..76c4902c76 100644 --- a/flow/pua/flatbuffers_builder.go +++ b/flow/pua/flatbuffers/flatbuffers_builder.go @@ -1,15 +1,17 @@ -package pua +package pua_flatbuffers import ( "slices" "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/pua" ) const VtableMetadataFields int = 2 type Builder struct { - ba BinaryArray + ba []byte vtables []int currentVT []int head int @@ -30,24 +32,24 @@ func (b *Builder) EndVector(ls *lua.LState, vectorSize int) int { } func (b *Builder) Offset() int { - return len(b.ba.data) - b.head + return len(b.ba) - b.head } func (b *Builder) Pad(pad int) { if pad > 0 { b.head -= pad - b.ba.Pad(pad, b.head) + Pad(b.ba, pad, b.head) } } func (b *Builder) Place(ls *lua.LState, x lua.LValue, n N) { b.head -= int(n.width) - n.Pack(ls, b.ba.data[b.head:], x) + n.Pack(ls, b.ba[b.head:], x) } func (b *Builder) PlaceU64(u64 uint64, n N) { b.head -= int(n.width) - n.PackU64(b.ba.data[b.head:], u64) + n.PackU64(b.ba[b.head:], u64) } func (b *Builder) Prep(width uint8, additional int) { @@ -60,12 +62,12 @@ func (b *Builder) Prep(width uint8, additional int) { space := alignsize + int(width) + additional if b.head < space { - oldlen := len(b.ba.data) - newdata := slices.Grow(b.ba.data, space) + oldlen := len(b.ba) + newdata := slices.Grow(b.ba, space) newdata = newdata[:cap(newdata)] copy(newdata[len(newdata)-oldlen:], newdata[:oldlen]) b.head += len(newdata) - oldlen - b.ba.data = newdata + b.ba = newdata } b.Pad(alignsize) @@ -164,9 +166,9 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { var existingVtable int for i := len(b.vtables) - 1; i >= 0; i -= 1 { vt2Offset := b.vtables[i] - vt2Start := len(b.ba.data) - vt2Offset - vt2Len := uint16n.UnpackU64(b.ba.data[vt2Start:]) - vt2 := b.ba.data[vt2Start+VtableMetadataFields*2 : vt2Start+int(vt2Len)] + vt2Start := len(b.ba) - vt2Offset + vt2Len := uint16n.UnpackU64(b.ba[vt2Start:]) + vt2 := b.ba[vt2Start+VtableMetadataFields*2 : vt2Start+int(vt2Len)] if vtableEqual(b.currentVT, objectOffset, vt2) { existingVtable = vt2Offset break @@ -187,11 +189,11 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { b.PrependVOffsetT(uint16(len(b.currentVT)+VtableMetadataFields) * 2) newOffset := b.Offset() - int32n.PackU64(b.ba.data[len(b.ba.data)-objectOffset:], uint64(newOffset-objectOffset)) + int32n.PackU64(b.ba[len(b.ba)-objectOffset:], uint64(newOffset-objectOffset)) b.vtables = append(b.vtables, newOffset) } else { - b.head = len(b.ba.data) - objectOffset - int32n.PackU64(b.ba.data[b.head:], uint64(existingVtable-objectOffset)) + b.head = len(b.ba) - objectOffset + int32n.PackU64(b.ba[b.head:], uint64(existingVtable-objectOffset)) } if len(b.currentVT) != 0 { @@ -200,7 +202,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { return objectOffset } -var LuaBuilder = LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} +var LuaBuilder = pua.LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} func FlatBuffers_Builder_Loader(ls *lua.LState) int { m := ls.NewTable() @@ -269,7 +271,7 @@ func BuilderNew(ls *lua.LState) int { initialSize := int(ls.CheckNumber(1)) ls.Push(LuaBuilder.New(ls, &Builder{ - ba: BinaryArray{data: make([]byte, initialSize)}, + ba: make([]byte, initialSize), vtables: make([]int, 0, 4), currentVT: make([]int, 0, 4), head: initialSize, @@ -291,16 +293,16 @@ func BuilderClear(ls *lua.LState) int { } b.currentVT = b.currentVT[:0] b.objectEnd = 0 - b.head = len(b.ba.data) + b.head = len(b.ba) return 0 } func BuilderOutput(ls *lua.LState) int { b := LuaBuilder.StartMeta(ls) if lua.LVIsFalse(ls.Get(2)) { - ls.Push(lua.LString(b.ba.data[b.head:])) + ls.Push(lua.LString(b.ba[b.head:])) } else { - ls.Push(lua.LString(b.ba.data)) + ls.Push(lua.LString(b.ba)) } return 1 } @@ -419,7 +421,7 @@ func createBytesHelper(ls *lua.LState, addnul bool) int { b.Prep(4, lens) } b.head -= lens - copy(b.ba.data[b.head:], s) + copy(b.ba[b.head:], s) ls.Push(lua.LNumber(b.EndVector(ls, lens))) return 1 diff --git a/flow/pua/flatbuffers_numtypes.go b/flow/pua/flatbuffers/flatbuffers_numtypes.go similarity index 74% rename from flow/pua/flatbuffers_numtypes.go rename to flow/pua/flatbuffers/flatbuffers_numtypes.go index 32a23fc205..9d62f197aa 100644 --- a/flow/pua/flatbuffers_numtypes.go +++ b/flow/pua/flatbuffers/flatbuffers_numtypes.go @@ -1,4 +1,4 @@ -package pua +package pua_flatbuffers import ( "encoding/binary" @@ -6,6 +6,8 @@ import ( "strconv" "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/pua" ) // Minimal API implemented for generated code @@ -146,9 +148,9 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { case 8: u64 := binary.LittleEndian.Uint64(buf) if n.signed { - return LuaI64.New(ls, int64(u64)) + return pua.LuaI64.New(ls, int64(u64)) } else { - return LuaU64.New(ls, u64) + return pua.LuaU64.New(ls, u64) } } case tyfloat: @@ -165,62 +167,57 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { panic("invalid numeric metatype") } -var LuaN = LuaUserDataType[N]{Name: "flatbuffers_n"} +var LuaN = pua.LuaUserDataType[N]{Name: "flatbuffers_n"} func FlatBuffers_N_Loader(ls *lua.LState) int { - m := ls.NewTable() - + mtidx := ls.CreateTable(0, 1) + mtidx.RawSetString("Unpack", ls.NewFunction(NUnpack)) mt := LuaView.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(NIndex)) + mt.RawSetString("__index", mtidx) uint16ud := LuaN.New(ls, uint16n) uint32ud := LuaN.New(ls, uint32n) int32ud := LuaN.New(ls, int32n) - ls.SetField(m, "Uint8", LuaN.New(ls, uint8n)) - ls.SetField(m, "Uint16", uint16ud) - ls.SetField(m, "Uint32", uint32ud) - ls.SetField(m, "Uint64", LuaN.New(ls, uint64n)) - ls.SetField(m, "Int8", LuaN.New(ls, int8n)) - ls.SetField(m, "Int16", LuaN.New(ls, int16n)) - ls.SetField(m, "Int32", int32ud) - ls.SetField(m, "Int64", LuaN.New(ls, int64n)) - ls.SetField(m, "Float32", LuaN.New(ls, float32n)) - ls.SetField(m, "Float64", LuaN.New(ls, float64n)) - ls.SetField(m, "Bool", LuaN.New(ls, booln)) - - ls.SetField(m, "UOffsetT", uint32ud) - ls.SetField(m, "VOffsetT", uint16ud) - ls.SetField(m, "SOffsetT", int32ud) + m := ls.NewTable() + m.RawSetString("Uint8", LuaN.New(ls, uint8n)) + m.RawSetString("Uint16", uint16ud) + m.RawSetString("Uint32", uint32ud) + m.RawSetString("Uint64", LuaN.New(ls, uint64n)) + m.RawSetString("Int8", LuaN.New(ls, int8n)) + m.RawSetString("Int16", LuaN.New(ls, int16n)) + m.RawSetString("Int32", int32ud) + m.RawSetString("Int64", LuaN.New(ls, int64n)) + m.RawSetString("Float32", LuaN.New(ls, float32n)) + m.RawSetString("Float64", LuaN.New(ls, float64n)) + m.RawSetString("Bool", LuaN.New(ls, booln)) + + m.RawSetString("UOffsetT", uint32ud) + m.RawSetString("VOffsetT", uint16ud) + m.RawSetString("SOffsetT", int32ud) ls.Push(m) return 1 } -func NIndex(ls *lua.LState) int { - n, key := LuaN.StartIndex(ls) - if key == "Unpack" { - var buf []byte - switch v := ls.Get(1).(type) { - case lua.LString: - buf = []byte(v) - case *lua.LUserData: - ba, ok := v.Value.(BinaryArray) - if ok { - buf = ba.data - } else { - ls.RaiseError("Invalid buf userdata passed to unpack") - return 0 - } - default: - ls.RaiseError("Invalid buf passed to unpack") +func NUnpack(ls *lua.LState) int { + n := LuaN.StartMeta(ls) + pos := max(CheckOffset(ls, 2), 1) + var buf []byte + switch v := ls.Get(1).(type) { + case lua.LString: + buf = []byte(v[pos-1:]) + case *lua.LUserData: + if ba, ok := v.Value.([]byte); ok { + buf = ba[pos-1:] + } else { + ls.RaiseError("Invalid buf userdata passed to unpack") return 0 } - pos := max(CheckOffset(ls, 2), 1) - ls.Push(n.Unpack(ls, buf[pos-1:])) - return 1 - } else { - ls.RaiseError("Unsupported field on N: " + key) + default: + ls.RaiseError("Invalid buf passed to unpack") return 0 } + ls.Push(n.Unpack(ls, buf)) + return 1 } diff --git a/flow/pua/flatbuffers_view.go b/flow/pua/flatbuffers/flatbuffers_view.go similarity index 76% rename from flow/pua/flatbuffers_view.go rename to flow/pua/flatbuffers/flatbuffers_view.go index 985fa84f0d..ec796efd49 100644 --- a/flow/pua/flatbuffers_view.go +++ b/flow/pua/flatbuffers/flatbuffers_view.go @@ -1,24 +1,20 @@ -package pua +package pua_flatbuffers import ( "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/pua" ) type View struct { - ba BinaryArray + ba []byte pos int // 0-based offset vtable int // 0-based offset vtableEnd uint16 hasv bool } -/* -func (view *View) Get(ls *lua.LState, n N, offset int) lua.LValue { - return n.Unpack(ls, view.ba.data[offset-1:]) -} -*/ - -var LuaView = LuaUserDataType[*View]{Name: "flatbuffers_view"} +var LuaView = pua.LuaUserDataType[*View]{Name: "flatbuffers_view"} func CheckOffset(ls *lua.LState, idx int) int { num := ls.CheckNumber(idx) @@ -41,13 +37,13 @@ func FlatBuffers_View_Loader(ls *lua.LState) int { func ViewNew(ls *lua.LState) int { buf := ls.Get(1) - var ba BinaryArray + var ba []byte switch val := buf.(type) { case lua.LString: - ba = BinaryArray{data: []byte(val)} + ba = []byte(val) case *lua.LUserData: var ok bool - ba, ok = val.Value.(BinaryArray) + ba, ok = val.Value.([]byte) if !ok { ls.RaiseError("invalid buf userdata passed to view.New") return 0 @@ -96,12 +92,12 @@ func ViewIndex(ls *lua.LState) int { func (view *View) Offset(vtoff uint16) uint16 { if !view.hasv { - view.vtable = view.pos - int(int32(int32n.UnpackU64(view.ba.data[view.pos:]))) - view.vtableEnd = uint16(uint16n.UnpackU64(view.ba.data[view.vtable:])) + view.vtable = view.pos - int(int32(int32n.UnpackU64(view.ba[view.pos:]))) + view.vtableEnd = uint16(uint16n.UnpackU64(view.ba[view.vtable:])) view.hasv = true } if vtoff < view.vtableEnd { - return uint16(uint16n.UnpackU64(view.ba.data[view.vtable+int(vtoff):])) + return uint16(uint16n.UnpackU64(view.ba[view.vtable+int(vtoff):])) } else { return 0 } @@ -109,12 +105,12 @@ func (view *View) Offset(vtoff uint16) uint16 { func (view *View) Vector(off int) int { off += view.pos - return off + int(uint32n.UnpackU64(view.ba.data[off:])) + 4 + return off + int(uint32n.UnpackU64(view.ba[off:])) + 4 } func (view *View) VectorLen(off int) uint32 { - off += int(uint32n.UnpackU64(view.ba.data[view.pos+off:])) - return uint32(uint32n.UnpackU64(view.ba.data[off:])) + off += int(uint32n.UnpackU64(view.ba[view.pos+off:])) + return uint32(uint32n.UnpackU64(view.ba[off:])) } func ViewOffset(ls *lua.LState) int { @@ -127,17 +123,17 @@ func ViewOffset(ls *lua.LState) int { func ViewIndirect(ls *lua.LState) int { view := LuaView.StartMeta(ls) off := CheckOffset(ls, 2) - ls.Push(lua.LNumber(off + int(uint32n.UnpackU64(view.ba.data[off:])))) + ls.Push(lua.LNumber(off + int(uint32n.UnpackU64(view.ba[off:])))) return 1 } func ViewString(ls *lua.LState) int { view := LuaView.StartMeta(ls) off := CheckOffset(ls, 2) - off += int(uint32n.UnpackU64(view.ba.data[off:])) + off += int(uint32n.UnpackU64(view.ba[off:])) start := off + 4 - length := int(uint32n.UnpackU64(view.ba.data[off:])) - ls.Push(lua.LString(view.ba.data[start : start+length])) + length := int(uint32n.UnpackU64(view.ba[off:])) + ls.Push(lua.LString(view.ba[start : start+length])) return 1 } @@ -175,7 +171,7 @@ func ViewVectorAsString(ls *lua.LState) int { stop = int(view.VectorLen(int(o))) } a := view.Vector(int(o)) + start - ls.Push(lua.LString(view.ba.data[a : a+stop-start])) + ls.Push(lua.LString(view.ba[a : a+stop-start])) return 1 } @@ -184,7 +180,7 @@ func ViewUnion(ls *lua.LState) int { t2ud, t2 := LuaView.Check(ls, 2) off := CheckOffset(ls, 3) off += view.pos - t2.pos = off + int(uint32n.UnpackU64(view.ba.data[off:])) + t2.pos = off + int(uint32n.UnpackU64(view.ba[off:])) t2.ba = view.ba t2ud.Value = t2 return 0 @@ -194,7 +190,7 @@ func ViewGet(ls *lua.LState) int { view := LuaView.StartMeta(ls) _, n := LuaN.Check(ls, 2) off := CheckOffset(ls, 3) - ls.Push(n.Unpack(ls, view.ba.data[off-1:])) + ls.Push(n.Unpack(ls, view.ba[off-1:])) return 1 } @@ -207,7 +203,7 @@ func ViewGetSlot(ls *lua.LState) int { return 1 } _, validatorFlags := LuaN.Check(ls, 4) - ls.Push(validatorFlags.Unpack(ls, view.ba.data[view.pos+int(off):])) + ls.Push(validatorFlags.Unpack(ls, view.ba[view.pos+int(off):])) return 1 } From 93c5ccbe727f6bd643927e03a5562e7a8884435e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Mar 2024 05:37:53 +0000 Subject: [PATCH 13/31] lua: bit32 --- flow/go.mod | 1 + flow/go.sum | 2 ++ flow/pua/peerdb.go | 3 +++ 3 files changed, 6 insertions(+) diff --git a/flow/go.mod b/flow/go.mod index 433d8c912c..e151b67788 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -10,6 +10,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.4 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 github.com/ClickHouse/clickhouse-go/v2 v2.21.1 + github.com/PeerDB-io/gluabit32 v1.0.0 github.com/aws/aws-sdk-go-v2 v1.25.3 github.com/aws/aws-sdk-go-v2/config v1.27.7 github.com/aws/aws-sdk-go-v2/credentials v1.17.7 diff --git a/flow/go.sum b/flow/go.sum index 4a3616a854..ec8e3499f8 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -54,6 +54,8 @@ github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= +github.com/PeerDB-io/gluabit32 v1.0.0 h1:jn88j22LqiqDoS47LUnvk29hsFQE6yW0imSsHHiYsV8= +github.com/PeerDB-io/gluabit32 v1.0.0/go.mod h1:tsHStN1XG5uGVWEA8d/RameB7el3PE3sVkvk8e3+FJg= github.com/alecthomas/assert/v2 v2.6.0 h1:o3WJwILtexrEUk3cUVal3oiQY2tfgr/FHWiz/v2n4FU= github.com/alecthomas/assert/v2 v2.6.0/go.mod h1:Bze95FyfUr7x34QZrjL+XP+0qgp/zg8yS+TtBj1WA3k= github.com/alecthomas/repr v0.4.0 h1:GhI2A8MACjfegCPVq9f1FLvIBS+DrQ2KQBFZP1iFzXc= diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 44dade8a89..c81256ef3c 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -12,6 +12,7 @@ import ( "github.com/shopspring/decimal" "github.com/yuin/gopher-lua" + "github.com/PeerDB-io/gluabit32" "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -34,6 +35,8 @@ func RegisterTypes(ls *lua.LState) { loaders := ls.G.Registry.RawGetString("_LOADERS").(*lua.LTable) loaders.RawSetInt(2, ls.NewFunction(LoadPeerdbScript)) + ls.PreloadModule("bit32", bit32.Loader) + mt := LuaRecord.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaRecordIndex)) From b4b57b6314b4916039a122cf6350d2245f66c904 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Mar 2024 14:14:21 +0000 Subject: [PATCH 14/31] Remove FlatBuffer_ prefixes --- ...flatbuffers_binaryarray.go => binaryarray.go} | 2 +- .../{flatbuffers_builder.go => builder.go} | 2 +- flow/pua/flatbuffers/flatbuffers.go | 16 ++++++++-------- .../{flatbuffers_numtypes.go => numtypes.go} | 2 +- .../flatbuffers/{flatbuffers_view.go => view.go} | 2 +- 5 files changed, 12 insertions(+), 12 deletions(-) rename flow/pua/flatbuffers/{flatbuffers_binaryarray.go => binaryarray.go} (97%) rename flow/pua/flatbuffers/{flatbuffers_builder.go => builder.go} (99%) rename flow/pua/flatbuffers/{flatbuffers_numtypes.go => numtypes.go} (99%) rename flow/pua/flatbuffers/{flatbuffers_view.go => view.go} (98%) diff --git a/flow/pua/flatbuffers/flatbuffers_binaryarray.go b/flow/pua/flatbuffers/binaryarray.go similarity index 97% rename from flow/pua/flatbuffers/flatbuffers_binaryarray.go rename to flow/pua/flatbuffers/binaryarray.go index caa087c7fc..8fd9b88d8a 100644 --- a/flow/pua/flatbuffers/flatbuffers_binaryarray.go +++ b/flow/pua/flatbuffers/binaryarray.go @@ -8,7 +8,7 @@ import ( var LuaBinaryArray = pua.LuaUserDataType[[]byte]{Name: "flatbuffers_binaryarray"} -func FlatBuffers_BinaryArray_Loader(ls *lua.LState) int { +func BinaryArray_Loader(ls *lua.LState) int { m := ls.NewTable() ls.SetField(m, "New", ls.NewFunction(BinaryArrayNew)) diff --git a/flow/pua/flatbuffers/flatbuffers_builder.go b/flow/pua/flatbuffers/builder.go similarity index 99% rename from flow/pua/flatbuffers/flatbuffers_builder.go rename to flow/pua/flatbuffers/builder.go index 76c4902c76..cc5f7882d3 100644 --- a/flow/pua/flatbuffers/flatbuffers_builder.go +++ b/flow/pua/flatbuffers/builder.go @@ -204,7 +204,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { var LuaBuilder = pua.LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} -func FlatBuffers_Builder_Loader(ls *lua.LState) int { +func Builder_Loader(ls *lua.LState) int { m := ls.NewTable() m.RawSetString("New", ls.NewFunction(BuilderNew)) diff --git a/flow/pua/flatbuffers/flatbuffers.go b/flow/pua/flatbuffers/flatbuffers.go index 2163417d30..43404b2cda 100644 --- a/flow/pua/flatbuffers/flatbuffers.go +++ b/flow/pua/flatbuffers/flatbuffers.go @@ -5,29 +5,29 @@ import ( ) func Loader(ls *lua.LState) int { - ls.PreloadModule("flatbuffers.binaryarray", FlatBuffers_BinaryArray_Loader) - ls.PreloadModule("flatbuffers.builder", FlatBuffers_Builder_Loader) - ls.PreloadModule("flatbuffers.numTypes", FlatBuffers_N_Loader) - ls.PreloadModule("flatbuffers.view", FlatBuffers_View_Loader) + ls.PreloadModule("flatbuffers.binaryarray", BinaryArray_Loader) + ls.PreloadModule("flatbuffers.builder", Builder_Loader) + ls.PreloadModule("flatbuffers.numTypes", N_Loader) + ls.PreloadModule("flatbuffers.view", View_Loader) m := ls.NewTable() - ls.Push(ls.NewFunction(FlatBuffers_N_Loader)) + ls.Push(ls.NewFunction(N_Loader)) ls.Call(0, 1) m.RawSetString("N", ls.Get(-1)) ls.Pop(1) - ls.Push(ls.NewFunction(FlatBuffers_View_Loader)) + ls.Push(ls.NewFunction(View_Loader)) ls.Call(0, 1) m.RawSetString("view", ls.Get(-1)) ls.Pop(1) - ls.Push(ls.NewFunction(FlatBuffers_BinaryArray_Loader)) + ls.Push(ls.NewFunction(BinaryArray_Loader)) ls.Call(0, 1) m.RawSetString("binaryArray", ls.Get(-1)) ls.Pop(1) - ls.Push(ls.NewFunction(FlatBuffers_Builder_Loader)) + ls.Push(ls.NewFunction(Builder_Loader)) ls.Call(0, 1) builder := ls.GetField(ls.Get(-1), "New") m.RawSetString("Builder", builder) diff --git a/flow/pua/flatbuffers/flatbuffers_numtypes.go b/flow/pua/flatbuffers/numtypes.go similarity index 99% rename from flow/pua/flatbuffers/flatbuffers_numtypes.go rename to flow/pua/flatbuffers/numtypes.go index 9d62f197aa..2de312985a 100644 --- a/flow/pua/flatbuffers/flatbuffers_numtypes.go +++ b/flow/pua/flatbuffers/numtypes.go @@ -169,7 +169,7 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { var LuaN = pua.LuaUserDataType[N]{Name: "flatbuffers_n"} -func FlatBuffers_N_Loader(ls *lua.LState) int { +func N_Loader(ls *lua.LState) int { mtidx := ls.CreateTable(0, 1) mtidx.RawSetString("Unpack", ls.NewFunction(NUnpack)) mt := LuaView.NewMetatable(ls) diff --git a/flow/pua/flatbuffers/flatbuffers_view.go b/flow/pua/flatbuffers/view.go similarity index 98% rename from flow/pua/flatbuffers/flatbuffers_view.go rename to flow/pua/flatbuffers/view.go index ec796efd49..9eed2ef3a2 100644 --- a/flow/pua/flatbuffers/flatbuffers_view.go +++ b/flow/pua/flatbuffers/view.go @@ -24,7 +24,7 @@ func CheckOffset(ls *lua.LState, idx int) int { return int(num) } -func FlatBuffers_View_Loader(ls *lua.LState) int { +func View_Loader(ls *lua.LState) int { m := ls.NewTable() ls.SetField(m, "New", ls.NewFunction(ViewNew)) From 3b2e236b24b693b9df33d27658f265e460b64e77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Mar 2024 14:20:25 +0000 Subject: [PATCH 15/31] adjust arrays to 1 based offset when row value is an array being converted to table --- flow/pua/peerdb.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index c81256ef3c..9f72df00ec 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -224,7 +224,7 @@ func LuaRecordIndex(ls *lua.LState) int { func qvToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) *lua.LTable { tbl := ls.CreateTable(len(s), 0) for idx, val := range s { - tbl.RawSetInt(idx, f(val)) + tbl.RawSetInt(idx+1, f(val)) } return tbl } From 9ec1c584c513fa7f21c2ae58782ae1fdf04791a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Mar 2024 14:33:31 +0000 Subject: [PATCH 16/31] Add lo/hi properties to i64/u64 types to help use with bit32 library --- flow/pua/peerdb.go | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 9f72df00ec..a2e8e906cc 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -467,6 +467,10 @@ func LuaI64Index(ls *lua.LState) int { ls.Push(LuaU64.New(ls, uint64(i64))) case "float64": ls.Push(lua.LNumber(i64)) + case "hi": + ls.Push(lua.LNumber(i64 >> 32)) + case "lo": + ls.Push(lua.LNumber(uint32(i64))) default: return 0 } @@ -483,6 +487,10 @@ func LuaU64Index(ls *lua.LState) int { ls.Push(u64ud) case "float64": ls.Push(lua.LNumber(u64)) + case "hi": + ls.Push(lua.LNumber(u64 >> 32)) + case "lo": + ls.Push(lua.LNumber(uint32(u64))) default: return 0 } @@ -590,8 +598,7 @@ func LuaDecimalIndex(ls *lua.LState) int { case "int64": ls.Push(LuaI64.New(ls, num.IntPart())) case "float64": - f64, _ := num.Float64() - ls.Push(lua.LNumber(f64)) + ls.Push(lua.LNumber(num.InexactFloat64())) default: return 0 } From 163a1e251ee1bd948f5b032d957cb25c5065e69e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Mar 2024 23:08:52 +0000 Subject: [PATCH 17/31] onRecord: support for multiple returns onRecord: support for specifying non-value fields on record ui: add partitioner option also remove dofile/loadfile --- flow/connectors/kafka/kafka.go | 109 +++++++++++++++--- flow/pua/peerdb.go | 3 + nexus/analyzer/src/lib.rs | 14 ++- protos/peers.proto | 1 + .../peers/create/[peerType]/helpers/common.ts | 2 + ui/app/peers/create/[peerType]/helpers/ka.ts | 23 ++++ ui/app/peers/create/[peerType]/schema.ts | 27 +++-- ui/components/PeerForms/KafkaConfig.tsx | 12 +- 8 files changed, 154 insertions(+), 37 deletions(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 5bcfe839a6..fe9c397c3b 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -8,6 +8,7 @@ import ( "log/slog" "strings" "sync" + "unsafe" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/sasl/plain" @@ -31,12 +32,37 @@ type KafkaConnector struct { logger log.Logger } +func unsafeFastStringToReadOnlyBytes(s string) []byte { + return unsafe.Slice(unsafe.StringData(s), len(s)) +} + +func LVAsReadOnlyBytes(ls *lua.LState, v lua.LValue) ([]byte, error) { + str, err := LVAsStringOrNil(ls, v) + if err != nil { + return nil, err + } else if str == "" { + return nil, nil + } else { + return unsafeFastStringToReadOnlyBytes(str), nil + } +} + +func LVAsStringOrNil(ls *lua.LState, v lua.LValue) (string, error) { + if lstr, ok := v.(lua.LString); ok { + return string(lstr), nil + } else if v == lua.LNil { + return "", nil + } else { + return "", fmt.Errorf("invalid bytes, must be nil or string: %s", v) + } +} + func NewKafkaConnector( ctx context.Context, config *protos.KafkaConfig, ) (*KafkaConnector, error) { optionalOpts := append( - make([]kgo.Opt, 0, 6), + make([]kgo.Opt, 0, 7), kgo.SeedBrokers(config.Servers...), kgo.AllowAutoTopicCreation(), kgo.WithLogger(kslog.New(slog.Default())), // TODO use logger.LoggerFromCtx @@ -45,6 +71,18 @@ func NewKafkaConnector( if !config.DisableTls { optionalOpts = append(optionalOpts, kgo.DialTLSConfig(&tls.Config{MinVersion: tls.VersionTLS13})) } + switch config.Partitioner { + case "LeastBackup": + optionalOpts = append(optionalOpts, kgo.RecordPartitioner(kgo.LeastBackupPartitioner())) + case "Manual": + optionalOpts = append(optionalOpts, kgo.RecordPartitioner(kgo.ManualPartitioner())) + case "RoundRobin": + optionalOpts = append(optionalOpts, kgo.RecordPartitioner(kgo.RoundRobinPartitioner())) + case "StickyKey": + optionalOpts = append(optionalOpts, kgo.RecordPartitioner(kgo.StickyKeyPartitioner(nil))) + case "Sticky": + optionalOpts = append(optionalOpts, kgo.RecordPartitioner(kgo.StickyPartitioner())) + } if config.Username != "" { switch config.Sasl { case "PLAIN": @@ -194,24 +232,69 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords } ls.Push(fn) ls.Push(pua.LuaRecord.New(ls, record)) - err := ls.PCall(1, 1, nil) + err := ls.PCall(1, -1, nil) if err != nil { return nil, fmt.Errorf("script failed: %w", err) } - value := ls.Get(-1) - ls.SetTop(0) - if value != lua.LNil { - lstr, ok := value.(lua.LString) - if !ok { - return nil, fmt.Errorf("script returned non-nil non-string: %s", value) + args := ls.GetTop() + for i := range args { + value := ls.Get(i - args) + var kr *kgo.Record + switch v := value.(type) { + case lua.LString: + kr = kgo.StringRecord(string(v)) + case *lua.LTable: + key, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "key")) + if err != nil { + return nil, fmt.Errorf("invalid key, %w", err) + } + value, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "value")) + if err != nil { + return nil, fmt.Errorf("invalid value, %w", err) + } + topic, err := LVAsStringOrNil(ls, ls.GetField(v, "topic")) + if err != nil { + return nil, fmt.Errorf("invalid topic, %w", err) + } + partition := int32(lua.LVAsNumber(ls.GetField(v, "partition"))) + kr = &kgo.Record{ + Key: key, + Value: value, + Topic: topic, + Partition: partition, + } + lheaders := ls.GetField(v, "headers") + if headers, ok := lheaders.(*lua.LTable); ok { + headers.ForEach(func(k, v lua.LValue) { + kstr := k.String() + vbytes, err := LVAsReadOnlyBytes(ls, v) + if err != nil { + vbytes = unsafeFastStringToReadOnlyBytes(err.Error()) + } + kr.Headers = append(kr.Headers, kgo.RecordHeader{ + Key: kstr, + Value: vbytes, + }) + }) + } else if lua.LVAsBool(lheaders) { + return nil, fmt.Errorf("invalid headers, must be nil or table: %s", lheaders) + } + case *lua.LNilType: + default: + return nil, fmt.Errorf("script returned invalid value: %s", value) } + if kr != nil { + if kr.Topic == "" { + kr.Topic = record.GetDestinationTableName() + } - wg.Add(1) - topic := record.GetDestinationTableName() - c.client.Produce(wgCtx, &kgo.Record{Topic: topic, Value: []byte(lstr)}, produceCb) - numRecords += 1 - tableNameRowsMapping[topic] += 1 + wg.Add(1) + c.client.Produce(wgCtx, kr, produceCb) + numRecords += 1 + tableNameRowsMapping[kr.Topic] += 1 + } } + ls.SetTop(0) } waitChan := make(chan struct{}) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index a2e8e906cc..bb824d8313 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -30,6 +30,9 @@ var ( ) func RegisterTypes(ls *lua.LState) { + ls.Env.RawSetString("loadfile", lua.LNil) + ls.Env.RawSetString("dofile", lua.LNil) + // gopher-lua provides 2 loaders {preload, file} // overwrite file loader with one retrieving scripts from database loaders := ls.G.Registry.RawGetString("_LOADERS").(*lua.LTable) diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 02d5e8223f..3213b50ec0 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -830,15 +830,23 @@ fn parse_db_options( .collect::>(), username: opts .get("user") - .context("no username specified")? + .cloned() + .unwrap_or_default() .to_string(), password: opts .get("password") - .context("no password specified")? + .cloned() + .unwrap_or_default() .to_string(), sasl: opts .get("sasl_mechanism") - .context("no sasl mechanism specified")? + .cloned() + .unwrap_or_default() + .to_string(), + partitioner: opts + .get("sasl_mechanism") + .cloned() + .unwrap_or_default() .to_string(), disable_tls: opts .get("disable_tls") diff --git a/protos/peers.proto b/protos/peers.proto index a4c9fbc79d..12fc88bfb3 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -114,6 +114,7 @@ message KafkaConfig { string password = 3; string sasl = 4; bool disable_tls = 5; + string partitioner = 6; } enum DBType { diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index ba14dafb2f..e418fd9266 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -13,6 +13,8 @@ export interface PeerSetting { tips?: string; helpfulLink?: string; default?: string | number; + placeholder?: string; + options?: { value: string; label: string }[]; } export const getBlankSetting = (dbType: string): PeerConfig => { diff --git a/ui/app/peers/create/[peerType]/helpers/ka.ts b/ui/app/peers/create/[peerType]/helpers/ka.ts index 4617e7360b..5091d29e7e 100644 --- a/ui/app/peers/create/[peerType]/helpers/ka.ts +++ b/ui/app/peers/create/[peerType]/helpers/ka.ts @@ -28,8 +28,30 @@ export const kaSetting: PeerSetting[] = [ stateHandler: (value, setter) => setter((curr) => ({ ...curr, sasl: value as string })), type: 'select', + placeholder: 'Select a mechanism', helpfulLink: 'https://docs.redpanda.com/current/manage/security/authentication/#scram', + options: [ + { value: 'PLAIN', label: 'PLAIN' }, + { value: 'SCRAM-SHA-256', label: 'SCRAM-SHA-256' }, + { value: 'SCRAM-SHA-512', label: 'SCRAM-SHA-512' }, + ], + }, + { + label: 'Partitioner', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, partitioner: value as string })), + type: 'select', + placeholder: 'Select a partitioner', + helpfulLink: + 'https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo#Partitioner', + options: [ + { value: 'LeastBackup', label: 'Least Backup' }, + { value: 'Manual', label: 'Manual' }, + { value: 'RoundRobin', label: 'Round Robin' }, + { value: 'StickyKey', label: 'Sticky Key' }, + { value: 'Sticky', label: 'Sticky' }, + ], }, { label: 'Disable TLS?', @@ -46,5 +68,6 @@ export const blankKaSetting: KafkaConfig = { username: '', password: '', sasl: 'PLAIN', + partitioner: '', disableTls: false, }; diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index f9541ac274..117b8738fe 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -288,18 +288,21 @@ export const kaSchema = z.object({ username: z.string().optional(), password: z.string().optional(), sasl: z - .union( - [ - z.literal('PLAIN'), - z.literal('SCRAM-SHA-256'), - z.literal('SCRAM-SHA-512'), - ], - { - errorMap: () => ({ - message: 'One of PLAIN, SCRAM-SHA-256, or SCRAM-SHA-512 is required.', - }), - } - ) + .union([ + z.literal('PLAIN'), + z.literal('SCRAM-SHA-256'), + z.literal('SCRAM-SHA-512'), + ]) + .optional(), + partitioner: z + .union([ + z.literal('Default'), + z.literal('LeastBackup'), + z.literal('Manual'), + z.literal('RoundRobin'), + z.literal('StickyKey'), + z.literal('Sticky'), + ]) .optional(), disableTls: z.boolean().optional(), }); diff --git a/ui/components/PeerForms/KafkaConfig.tsx b/ui/components/PeerForms/KafkaConfig.tsx index 0e0549429a..65df972ba5 100644 --- a/ui/components/PeerForms/KafkaConfig.tsx +++ b/ui/components/PeerForms/KafkaConfig.tsx @@ -13,12 +13,6 @@ interface KafkaProps { setter: PeerSetter; } -const saslOptions = [ - { value: 'PLAIN', label: 'PLAIN' }, - { value: 'SCRAM-SHA-256', label: 'SCRAM-SHA-256' }, - { value: 'SCRAM-SHA-512', label: 'SCRAM-SHA-512' }, -]; - const KafkaForm = ({ setter }: KafkaProps) => { return (
@@ -55,15 +49,15 @@ const KafkaForm = ({ setter }: KafkaProps) => { /> ) : setting.type === 'select' ? ( SASL Mechanism} + label={} action={ val && setting.stateHandler(val.value, setter) } - options={saslOptions} + options={setting.options} theme={SelectTheme} /> } From 5554b511fd76fb84ba4bccf830c2cee7967ee72b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 03:28:39 +0000 Subject: [PATCH 18/31] flatbuffers: license directory as ApacheV2 & add readme explaining --- flow/pua/flatbuffers/LICENSE | 203 +++++++++++++++++++++++++++++++++ flow/pua/flatbuffers/readme.md | 6 + 2 files changed, 209 insertions(+) create mode 100644 flow/pua/flatbuffers/LICENSE create mode 100644 flow/pua/flatbuffers/readme.md diff --git a/flow/pua/flatbuffers/LICENSE b/flow/pua/flatbuffers/LICENSE new file mode 100644 index 0000000000..6b0b1270ff --- /dev/null +++ b/flow/pua/flatbuffers/LICENSE @@ -0,0 +1,203 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + diff --git a/flow/pua/flatbuffers/readme.md b/flow/pua/flatbuffers/readme.md new file mode 100644 index 0000000000..494f17e6d3 --- /dev/null +++ b/flow/pua/flatbuffers/readme.md @@ -0,0 +1,6 @@ +This directory is licensed under Apache V2 the files are a derivative work of https://github.com/google/flatbuffers/tree/master/lua + +This Go port exists for 3 reasons: +1. upstream lua generator assumes 5.3 (relies on string.unpack/string.pack/integer) despite compatibiltiy shims. gopher-lua is 5.1+goto. So need the code to work with 64 bit integers while Lua 5.1 numbers are all float64 +2. their lua runtime library has some flaws, like encoding bool as ascii "0" or "1" instead of "\0" or "\1" +3. last & definitely least, perf From 188a30db61161ea3492cb42e37512539afa1b965 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 15:06:15 +0000 Subject: [PATCH 19/31] Rename LuaUserDataType to UserDataType --- flow/pua/flatbuffers/binaryarray.go | 2 +- flow/pua/flatbuffers/builder.go | 2 +- flow/pua/flatbuffers/numtypes.go | 2 +- flow/pua/flatbuffers/view.go | 2 +- flow/pua/peerdb.go | 16 ++++++++-------- flow/pua/userdata.go | 14 +++++++------- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/flow/pua/flatbuffers/binaryarray.go b/flow/pua/flatbuffers/binaryarray.go index 8fd9b88d8a..c51666f0f5 100644 --- a/flow/pua/flatbuffers/binaryarray.go +++ b/flow/pua/flatbuffers/binaryarray.go @@ -6,7 +6,7 @@ import ( "github.com/PeerDB-io/peer-flow/pua" ) -var LuaBinaryArray = pua.LuaUserDataType[[]byte]{Name: "flatbuffers_binaryarray"} +var LuaBinaryArray = pua.UserDataType[[]byte]{Name: "flatbuffers_binaryarray"} func BinaryArray_Loader(ls *lua.LState) int { m := ls.NewTable() diff --git a/flow/pua/flatbuffers/builder.go b/flow/pua/flatbuffers/builder.go index cc5f7882d3..0b4ab18eb8 100644 --- a/flow/pua/flatbuffers/builder.go +++ b/flow/pua/flatbuffers/builder.go @@ -202,7 +202,7 @@ func (b *Builder) WriteVtable(ls *lua.LState) int { return objectOffset } -var LuaBuilder = pua.LuaUserDataType[*Builder]{Name: "flatbuffers_builder"} +var LuaBuilder = pua.UserDataType[*Builder]{Name: "flatbuffers_builder"} func Builder_Loader(ls *lua.LState) int { m := ls.NewTable() diff --git a/flow/pua/flatbuffers/numtypes.go b/flow/pua/flatbuffers/numtypes.go index 2de312985a..f3ff6c6f6a 100644 --- a/flow/pua/flatbuffers/numtypes.go +++ b/flow/pua/flatbuffers/numtypes.go @@ -167,7 +167,7 @@ func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { panic("invalid numeric metatype") } -var LuaN = pua.LuaUserDataType[N]{Name: "flatbuffers_n"} +var LuaN = pua.UserDataType[N]{Name: "flatbuffers_n"} func N_Loader(ls *lua.LState) int { mtidx := ls.CreateTable(0, 1) diff --git a/flow/pua/flatbuffers/view.go b/flow/pua/flatbuffers/view.go index 9eed2ef3a2..f5b5e698a0 100644 --- a/flow/pua/flatbuffers/view.go +++ b/flow/pua/flatbuffers/view.go @@ -14,7 +14,7 @@ type View struct { hasv bool } -var LuaView = pua.LuaUserDataType[*View]{Name: "flatbuffers_view"} +var LuaView = pua.UserDataType[*View]{Name: "flatbuffers_view"} func CheckOffset(ls *lua.LState, idx int) int { num := ls.CheckNumber(idx) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index bb824d8313..8b611af3f9 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -19,14 +19,14 @@ import ( ) var ( - LuaRecord = LuaUserDataType[model.Record]{Name: "peerdb_record"} - LuaRow = LuaUserDataType[*model.RecordItems]{Name: "peerdb_row"} - LuaI64 = LuaUserDataType[int64]{Name: "flatbuffers_i64"} - LuaU64 = LuaUserDataType[uint64]{Name: "flatbuffers_u64"} - LuaTime = LuaUserDataType[time.Time]{Name: "peerdb_time"} - LuaUuid = LuaUserDataType[uuid.UUID]{Name: "peerdb_uuid"} - LuaBigInt = LuaUserDataType[*big.Int]{Name: "peerdb_bigint"} - LuaDecimal = LuaUserDataType[decimal.Decimal]{Name: "peerdb_bigrat"} + LuaRecord = UserDataType[model.Record]{Name: "peerdb_record"} + LuaRow = UserDataType[*model.RecordItems]{Name: "peerdb_row"} + LuaI64 = UserDataType[int64]{Name: "flatbuffers_i64"} + LuaU64 = UserDataType[uint64]{Name: "flatbuffers_u64"} + LuaTime = UserDataType[time.Time]{Name: "peerdb_time"} + LuaUuid = UserDataType[uuid.UUID]{Name: "peerdb_uuid"} + LuaBigInt = UserDataType[*big.Int]{Name: "peerdb_bigint"} + LuaDecimal = UserDataType[decimal.Decimal]{Name: "peerdb_bigrat"} ) func RegisterTypes(ls *lua.LState) { diff --git a/flow/pua/userdata.go b/flow/pua/userdata.go index 5b87881fa8..e0dba8254f 100644 --- a/flow/pua/userdata.go +++ b/flow/pua/userdata.go @@ -4,9 +4,9 @@ import ( "github.com/yuin/gopher-lua" ) -type LuaUserDataType[T any] struct{ Name string } +type UserDataType[T any] struct{ Name string } -func (udt *LuaUserDataType[T]) New(ls *lua.LState, val T) *lua.LUserData { +func (udt *UserDataType[T]) New(ls *lua.LState, val T) *lua.LUserData { return &lua.LUserData{ Value: val, Env: ls.Env, @@ -14,15 +14,15 @@ func (udt *LuaUserDataType[T]) New(ls *lua.LState, val T) *lua.LUserData { } } -func (udt *LuaUserDataType[T]) NewMetatable(ls *lua.LState) *lua.LTable { +func (udt *UserDataType[T]) NewMetatable(ls *lua.LState) *lua.LTable { return ls.NewTypeMetatable(udt.Name) } -func (udt *LuaUserDataType[T]) Metatable(ls *lua.LState) lua.LValue { +func (udt *UserDataType[T]) Metatable(ls *lua.LState) lua.LValue { return ls.GetTypeMetatable(udt.Name) } -func (udt *LuaUserDataType[T]) Check(ls *lua.LState, idx int) (*lua.LUserData, T) { +func (udt *UserDataType[T]) Check(ls *lua.LState, idx int) (*lua.LUserData, T) { ud := ls.CheckUserData(idx) val, ok := ud.Value.(T) if !ok { @@ -31,11 +31,11 @@ func (udt *LuaUserDataType[T]) Check(ls *lua.LState, idx int) (*lua.LUserData, T return ud, val } -func (udt *LuaUserDataType[T]) StartMeta(ls *lua.LState) T { +func (udt *UserDataType[T]) StartMeta(ls *lua.LState) T { _, val := udt.Check(ls, 1) return val } -func (udt *LuaUserDataType[T]) StartIndex(ls *lua.LState) (T, string) { +func (udt *UserDataType[T]) StartIndex(ls *lua.LState) (T, string) { return udt.StartMeta(ls), ls.CheckString(2) } From f5c16684161f8237e26d87d326cff78571e92216 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 17:29:44 +0000 Subject: [PATCH 20/31] add some basic unit testing to pua, fix i64/u64 comparisons --- flow/pua/peerdb.go | 15 +++++---- flow/pua/peerdb_test.go | 67 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 76 insertions(+), 6 deletions(-) create mode 100644 flow/pua/peerdb_test.go diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 8b611af3f9..0044e56371 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -51,19 +51,22 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaUuidString)) + eq64 := ls.NewFunction(Lua64Eq) + le64 := ls.NewFunction(Lua64Le) + lt64 := ls.NewFunction(Lua64Lt) mt = LuaI64.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaI64Index)) mt.RawSetString("__tostring", ls.NewFunction(LuaI64String)) - mt.RawSetString("__eq", ls.NewFunction(Lua64Eq)) - mt.RawSetString("__le", ls.NewFunction(Lua64Le)) - mt.RawSetString("__lt", ls.NewFunction(Lua64Lt)) + mt.RawSetString("__eq", eq64) + mt.RawSetString("__le", le64) + mt.RawSetString("__lt", lt64) mt = LuaU64.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaU64Index)) mt.RawSetString("__tostring", ls.NewFunction(LuaU64String)) - mt.RawSetString("__eq", ls.NewFunction(Lua64Eq)) - mt.RawSetString("__le", ls.NewFunction(Lua64Le)) - mt.RawSetString("__lt", ls.NewFunction(Lua64Lt)) + mt.RawSetString("__eq", eq64) + mt.RawSetString("__le", le64) + mt.RawSetString("__lt", lt64) mt = LuaTime.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaTimeIndex)) diff --git a/flow/pua/peerdb_test.go b/flow/pua/peerdb_test.go new file mode 100644 index 0000000000..fdbcd53295 --- /dev/null +++ b/flow/pua/peerdb_test.go @@ -0,0 +1,67 @@ +package pua + +import ( + "testing" + + "github.com/google/uuid" + "github.com/yuin/gopher-lua" +) + +func assert(t *testing.T, ls *lua.LState, source string) { + t.Helper() + err := ls.DoString(source) + if err != nil { + t.Log(err) + t.FailNow() + } +} + +func Test_Lua(t *testing.T) { + t.Parallel() + + ls := lua.NewState(lua.Options{}) + RegisterTypes(ls) + + id := uuid.UUID([16]byte{2, 3, 5, 7, 11, 13, 17, 19, 127, 131, 137, 139, 149, 151, 241, 251}) + ls.Env.RawSetString("uuid", LuaUuid.New(ls, id)) + + n5 := int64(-5) + ls.Env.RawSetString("i64p5", LuaI64.New(ls, 5)) + ls.Env.RawSetString("i64p5_2", LuaI64.New(ls, 5)) + ls.Env.RawSetString("u64p5", LuaU64.New(ls, 5)) + ls.Env.RawSetString("u64p5_2", LuaU64.New(ls, 5)) + ls.Env.RawSetString("i64n5", LuaI64.New(ls, n5)) + ls.Env.RawSetString("u64n5", LuaU64.New(ls, uint64(n5))) + + assert(t, ls, ` +assert(require('bit32').band(173, 21) == 5) +assert(dofile == nil) +assert(loadfile == nil) + +assert(uuid[0] == 2) +assert(uuid[1] == 3) +assert(uuid[2] == 5) +assert(uuid[3] == 7) +assert(uuid[4] == 11) +assert(uuid[5] == 13) +assert(uuid[6] == 17) +assert(uuid[7] == 19) +assert(uuid[8] == 127) +assert(uuid[9] == 131) +assert(uuid[10] == 137) +assert(uuid[11] == 139) +assert(uuid[12] == 149) +assert(uuid[13] == 151) +assert(uuid[14] == 241) +assert(uuid[15] == 251) + +print(i64p5, u64p5) +assert(i64p5 == u64p5) +assert(i64p5 ~= i64n5) +assert(i64n5 ~= u64n5) +assert(i64p5 == i64p5_2) +assert(u64p5 == u64p5_2) +assert(u64n5 > i64p5) +assert(i64p5 > i64n5) +`) +} From 820cd236fa5b84774d5ff7cb12a741b63293cbc3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 19:17:14 +0000 Subject: [PATCH 21/31] first stab at a basic kafka e2e test --- .github/workflows/flow.yml | 5 ++ flow/e2e/kafka/kafka_test.go | 133 +++++++++++++++++++++++++++++++++++ 2 files changed, 138 insertions(+) create mode 100644 flow/e2e/kafka/kafka_test.go diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 38bc636f51..54919f0bde 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -86,6 +86,11 @@ jobs: env: PGPASSWORD: postgres + - name: start redpanda + uses: redpanda-data/github-action@v0.1.4 + with: + version: "latest" + - name: Install Temporal CLI uses: temporalio/setup-temporal@v0 diff --git a/flow/e2e/kafka/kafka_test.go b/flow/e2e/kafka/kafka_test.go new file mode 100644 index 0000000000..3670706281 --- /dev/null +++ b/flow/e2e/kafka/kafka_test.go @@ -0,0 +1,133 @@ +package e2e_kafka + +import ( + "context" + "fmt" + "strings" + "testing" + "time" + + "github.com/jackc/pgx/v5" + "github.com/stretchr/testify/require" + "github.com/twmb/franz-go/pkg/kgo" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" +) + +type KafkaSuite struct { + t *testing.T + conn *connpostgres.PostgresConnector + suffix string +} + +func (s KafkaSuite) T() *testing.T { + return s.t +} + +func (s KafkaSuite) Connector() *connpostgres.PostgresConnector { + return s.conn +} + +func (s KafkaSuite) Conn() *pgx.Conn { + return s.Connector().Conn() +} + +func (s KafkaSuite) Suffix() string { + return s.suffix +} + +func (s KafkaSuite) Peer() *protos.Peer { + return &protos.Peer{ + Name: e2e.AddSuffix(s, "kasimple"), + Type: protos.DBType_KAFKA, + Config: &protos.Peer_KafkaConfig{ + KafkaConfig: &protos.KafkaConfig{ + Servers: []string{"localhost:9092"}, + DisableTls: true, + }, + }, + } +} + +func (s KafkaSuite) DestinationTable(table string) string { + return table +} + +func (s KafkaSuite) Teardown() { + e2e.TearDownPostgres(s) +} + +func SetupSuite(t *testing.T) KafkaSuite { + t.Helper() + + suffix := "ka_" + strings.ToLower(shared.RandomString(8)) + conn, err := e2e.SetupPostgres(t, suffix) + require.NoError(t, err, "failed to setup postgres") + + return KafkaSuite{ + t: t, + conn: conn, + suffix: suffix, + } +} + +func Test_Kafka(t *testing.T) { + e2eshared.RunSuite(t, SetupSuite) +} + +func (s KafkaSuite) TestSimple() { + srcTableName := e2e.AttachSchema(s, "kasimple") + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + val text + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: srcTableName, + TableNameMapping: map[string]string{srcTableName: "katest"}, + Destination: s.Peer(), + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.Script = `function onRecord(r) return r.row.val end` + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id, val) VALUES (1, 'testval') + `, srcTableName)) + require.NoError(s.t, err) + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert", func() bool { + kafka, err := kgo.NewClient( + kgo.SeedBrokers("localhost:9092"), + kgo.ConsumeTopics("katest"), + ) + if err != nil { + return false + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + fetches := kafka.PollFetches(ctx) + fetches.EachTopic(func(ft kgo.FetchTopic) { + require.Equal(s.t, "katest", ft.Topic) + ft.EachRecord(func(r *kgo.Record) { + require.Equal(s.t, "testval", string(r.Value)) + }) + }) + return true + }) + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} From 00d40d4a4f31aa317160d8141a84092a2446a287 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 19:51:41 +0000 Subject: [PATCH 22/31] add CommitTime to cdc messages, expose to script --- flow/connectors/postgres/cdc.go | 41 +++++++++++++----- .../cdc_records/cdc_records_storage_test.go | 5 ++- flow/e2e/kafka/kafka_test.go | 8 +++- flow/model/model.go | 43 +++++++++---------- flow/pua/peerdb.go | 2 + 5 files changed, 62 insertions(+), 37 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index acf3ba7a40..8623b43aa0 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -35,7 +35,7 @@ type PostgresCDCSource struct { slot string publication string typeMap *pgtype.Map - commitLock bool + commitLock *pglogrepl.BeginMessage // for partitioned tables, maps child relid to parent relid childToParentRelIDMapping map[uint32]uint32 @@ -75,7 +75,7 @@ func (c *PostgresConnector) NewPostgresCDCSource(cdcConfig *PostgresCDCConfig) * publication: cdcConfig.Publication, childToParentRelIDMapping: cdcConfig.ChildToParentRelIDMap, typeMap: pgtype.NewMap(), - commitLock: false, + commitLock: nil, catalogPool: cdcConfig.CatalogPool, flowJobName: cdcConfig.FlowJobName, } @@ -188,7 +188,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco } } - if !p.commitLock { + if p.commitLock == nil { if cdcRecordsStorage.Len() >= int(req.MaxBatchSize) { return nil } @@ -208,7 +208,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco if !cdcRecordsStorage.IsEmpty() { p.logger.Info(fmt.Sprintf("standby deadline reached, have %d records", cdcRecordsStorage.Len())) - if !p.commitLock { + if p.commitLock == nil { p.logger.Info( fmt.Sprintf("no commit lock, returning currently accumulated records - %d", cdcRecordsStorage.Len())) @@ -241,7 +241,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return fmt.Errorf("consumeStream preempted: %w", ctxErr) } - if err != nil && !p.commitLock { + if err != nil && p.commitLock == nil { if pgconn.Timeout(err) { p.logger.Info(fmt.Sprintf("Stand-by deadline reached, returning currently accumulated records - %d", cdcRecordsStorage.Len())) @@ -408,6 +408,13 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco } } +func (p *PostgresCDCSource) commitTime() time.Time { + if p.commitLock != nil { + return p.commitLock.CommitTime + } + return time.Time{} +} + func (p *PostgresCDCSource) processMessage( ctx context.Context, batch *model.CDCRecordStream, @@ -423,7 +430,7 @@ func (p *PostgresCDCSource) processMessage( case *pglogrepl.BeginMessage: p.logger.Debug(fmt.Sprintf("BeginMessage => FinalLSN: %v, XID: %v", msg.FinalLSN, msg.Xid)) p.logger.Debug("Locking PullRecords at BeginMessage, awaiting CommitMessage") - p.commitLock = true + p.commitLock = msg case *pglogrepl.InsertMessage: return p.processInsertMessage(xld.WALStart, msg) case *pglogrepl.UpdateMessage: @@ -435,7 +442,7 @@ func (p *PostgresCDCSource) processMessage( p.logger.Debug(fmt.Sprintf("CommitMessage => CommitLSN: %v, TransactionEndLSN: %v", msg.CommitLSN, msg.TransactionEndLSN)) batch.UpdateLatestCheckpoint(int64(msg.CommitLSN)) - p.commitLock = false + p.commitLock = nil case *pglogrepl.RelationMessage: // treat all relation messages as corresponding to parent if partitioned. msg.RelationID = p.getParentRelIDIfPartitioned(msg.RelationID) @@ -483,7 +490,10 @@ func (p *PostgresCDCSource) processInsertMessage( } return &model.InsertRecord{ - CheckpointID: int64(lsn), + BaseRecord: model.BaseRecord{ + CheckpointID: int64(lsn), + CommitTime: p.commitTime(), + }, Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, SourceTableName: tableName, @@ -524,7 +534,10 @@ func (p *PostgresCDCSource) processUpdateMessage( } return &model.UpdateRecord{ - CheckpointID: int64(lsn), + BaseRecord: model.BaseRecord{ + CheckpointID: int64(lsn), + CommitTime: p.commitTime(), + }, OldItems: oldItems, NewItems: newItems, DestinationTableName: p.tableNameMapping[tableName].Name, @@ -561,7 +574,10 @@ func (p *PostgresCDCSource) processDeleteMessage( } return &model.DeleteRecord{ - CheckpointID: int64(lsn), + BaseRecord: model.BaseRecord{ + CheckpointID: int64(lsn), + CommitTime: p.commitTime(), + }, Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, SourceTableName: tableName, @@ -765,8 +781,11 @@ func (p *PostgresCDCSource) processRelationMessage( // only log audit if there is actionable delta if len(schemaDelta.AddedColumns) > 0 { rec := &model.RelationRecord{ + BaseRecord: model.BaseRecord{ + CheckpointID: int64(lsn), + CommitTime: p.commitTime(), + }, TableSchemaDelta: schemaDelta, - CheckpointID: int64(lsn), } return rec, p.auditSchemaDelta(ctx, p.flowJobName, rec) } diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go index 50796109f3..ba32b27b21 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go @@ -47,9 +47,12 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { PkeyColVal: [32]byte(pkeyColVal), } rec := &model.InsertRecord{ + BaseRecord: model.BaseRecord{ + CheckpointID: 1, + CommitTime: time.Now().UTC(), + }, SourceTableName: "test_src_tbl", DestinationTableName: "test_dst_tbl", - CheckpointID: 1, CommitID: 2, Items: &model.RecordItems{ ColToValIdx: map[string]int{ diff --git a/flow/e2e/kafka/kafka_test.go b/flow/e2e/kafka/kafka_test.go index 3670706281..f35e0f568a 100644 --- a/flow/e2e/kafka/kafka_test.go +++ b/flow/e2e/kafka/kafka_test.go @@ -91,13 +91,17 @@ func (s KafkaSuite) TestSimple() { `, srcTableName)) require.NoError(s.t, err) + _, err = s.Conn().Exec(context.Background(), `insert into public.scripts (name, lang, source) values + ('e2e_kasimple', 'lua', 'function onRecord(r) return r.row and r.row.val end') on conflict do nothing`) + require.NoError(s.t, err) + connectionGen := e2e.FlowConnectionGenerationConfig{ - FlowJobName: srcTableName, + FlowJobName: e2e.AddSuffix(s, "kasimple"), TableNameMapping: map[string]string{srcTableName: "katest"}, Destination: s.Peer(), } flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() - flowConnConfig.Script = `function onRecord(r) return r.row.val end` + flowConnConfig.Script = "e2e_kasimple" tc := e2e.NewTemporalClient(s.t) env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) diff --git a/flow/model/model.go b/flow/model/model.go index 06f56ec2f3..773be38e4c 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -49,6 +49,7 @@ type PullRecordsRequest struct { type Record interface { GetCheckpointID() int64 + GetCommitTime() time.Time GetDestinationTableName() string GetSourceTableName() string // get columns and values for the record @@ -74,23 +75,33 @@ func NewToJSONOptions(unnestCols []string, hstoreAsJSON bool) *ToJSONOptions { } } +type BaseRecord struct { + // CheckpointID is the ID of the record. + CheckpointID int64 `json:"checkpointId"` + // CommitTime from BeginMessage + CommitTime time.Time `json:"commitTime"` +} + +func (r *BaseRecord) GetCheckpointID() int64 { + return r.CheckpointID +} + +func (r *BaseRecord) GetCommitTime() time.Time { + return r.CommitTime +} + type InsertRecord struct { + BaseRecord // Name of the source table SourceTableName string // Name of the destination table DestinationTableName string - // CheckpointID is the ID of the record. - CheckpointID int64 // CommitID is the ID of the commit corresponding to this record. CommitID int64 // Items is a map of column name to value. Items *RecordItems } -func (r *InsertRecord) GetCheckpointID() int64 { - return r.CheckpointID -} - func (r *InsertRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -104,10 +115,9 @@ func (r *InsertRecord) GetItems() *RecordItems { } type UpdateRecord struct { + BaseRecord // Name of the source table SourceTableName string - // CheckpointID is the ID of the record. - CheckpointID int64 // Name of the destination table DestinationTableName string // OldItems is a map of column name to value. @@ -118,10 +128,6 @@ type UpdateRecord struct { UnchangedToastColumns map[string]struct{} } -func (r *UpdateRecord) GetCheckpointID() int64 { - return r.CheckpointID -} - func (r *UpdateRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -135,22 +141,17 @@ func (r *UpdateRecord) GetItems() *RecordItems { } type DeleteRecord struct { + BaseRecord // Name of the source table SourceTableName string // Name of the destination table DestinationTableName string - // CheckpointID is the ID of the record. - CheckpointID int64 // Items is a map of column name to value. Items *RecordItems // unchanged toast columns, filled from latest UpdateRecord UnchangedToastColumns map[string]struct{} } -func (r *DeleteRecord) GetCheckpointID() int64 { - return r.CheckpointID -} - func (r *DeleteRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -219,14 +220,10 @@ type NormalizeResponse struct { // being clever and passing the delta back as a regular record instead of heavy CDC refactoring. type RelationRecord struct { - CheckpointID int64 `json:"checkpointId"` + BaseRecord TableSchemaDelta *protos.TableSchemaDelta `json:"tableSchemaDelta"` } -func (r *RelationRecord) GetCheckpointID() int64 { - return r.CheckpointID -} - func (r *RelationRecord) GetDestinationTableName() string { return r.TableSchemaDelta.DstTableName } diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 0044e56371..313b4cacfb 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -217,6 +217,8 @@ func LuaRecordIndex(ls *lua.LState) int { } case "checkpoint": ls.Push(LuaI64.New(ls, record.GetCheckpointID())) + case "commit_time": + ls.Push(LuaTime.New(ls, record.GetCommitTime())) case "target": ls.Push(lua.LString(record.GetDestinationTableName())) case "source": From 876a79e9ae33dea471736a426c04db314847549a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Mar 2024 22:20:42 +0000 Subject: [PATCH 23/31] Add Script to mirror details in UI --- flow/pua/peerdb.go | 4 ++-- ui/app/mirrors/[mirrorId]/configValues.ts | 20 ++++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 313b4cacfb..92ef3cefb8 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -476,7 +476,7 @@ func LuaI64Index(ls *lua.LState) int { case "float64": ls.Push(lua.LNumber(i64)) case "hi": - ls.Push(lua.LNumber(i64 >> 32)) + ls.Push(lua.LNumber(uint32(i64 >> 32))) case "lo": ls.Push(lua.LNumber(uint32(i64))) default: @@ -496,7 +496,7 @@ func LuaU64Index(ls *lua.LState) int { case "float64": ls.Push(lua.LNumber(u64)) case "hi": - ls.Push(lua.LNumber(u64 >> 32)) + ls.Push(lua.LNumber(uint32(u64 >> 32))) case "lo": ls.Push(lua.LNumber(uint32(u64))) default: diff --git a/ui/app/mirrors/[mirrorId]/configValues.ts b/ui/app/mirrors/[mirrorId]/configValues.ts index 8d49f3090f..55b5a4c0b3 100644 --- a/ui/app/mirrors/[mirrorId]/configValues.ts +++ b/ui/app/mirrors/[mirrorId]/configValues.ts @@ -19,21 +19,21 @@ const MirrorValues = (mirrorConfig: FlowConnectionConfigs | undefined) => { label: 'Snapshot Parallel Tables', }, { - value: `${ - mirrorConfig?.cdcStagingPath?.length - ? mirrorConfig?.cdcStagingPath - : 'Local' - }`, + value: mirrorConfig?.cdcStagingPath || 'Local', label: 'CDC Staging Path', }, { - value: `${ - mirrorConfig?.snapshotStagingPath?.length - ? mirrorConfig?.snapshotStagingPath - : 'Local' - }`, + value: mirrorConfig?.snapshotStagingPath || 'Local', label: 'Snapshot Staging Path', }, + { + value: mirrorConfig?.snapshotStagingPath || 'Local', + label: 'Snapshot Staging Path', + }, + { + value: mirrorConfig?.script, + label: 'Script', + }, ]; }; export default MirrorValues; From 53028df0c4d11f17011ab04df8c5e0f7a5504ae2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 13:36:21 +0000 Subject: [PATCH 24/31] reduce BaseMessage size by 16 bytes only storing unix nano, good until year 2262 --- flow/connectors/postgres/cdc.go | 30 +++++++------------ .../cdc_records/cdc_records_storage_test.go | 4 +-- flow/model/model.go | 6 ++-- 3 files changed, 16 insertions(+), 24 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 8623b43aa0..e5bad73d62 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -408,11 +408,15 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco } } -func (p *PostgresCDCSource) commitTime() time.Time { +func (p *PostgresCDCSource) baseRecord(lsn pglogrepl.LSN) model.BaseRecord { + var nano int64 if p.commitLock != nil { - return p.commitLock.CommitTime + nano = p.commitLock.CommitTime.UnixNano() + } + return model.BaseRecord{ + CheckpointID: int64(lsn), + CommitTimeNano: nano, } - return time.Time{} } func (p *PostgresCDCSource) processMessage( @@ -490,10 +494,7 @@ func (p *PostgresCDCSource) processInsertMessage( } return &model.InsertRecord{ - BaseRecord: model.BaseRecord{ - CheckpointID: int64(lsn), - CommitTime: p.commitTime(), - }, + BaseRecord: p.baseRecord(lsn), Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, SourceTableName: tableName, @@ -534,10 +535,7 @@ func (p *PostgresCDCSource) processUpdateMessage( } return &model.UpdateRecord{ - BaseRecord: model.BaseRecord{ - CheckpointID: int64(lsn), - CommitTime: p.commitTime(), - }, + BaseRecord: p.baseRecord(lsn), OldItems: oldItems, NewItems: newItems, DestinationTableName: p.tableNameMapping[tableName].Name, @@ -574,10 +572,7 @@ func (p *PostgresCDCSource) processDeleteMessage( } return &model.DeleteRecord{ - BaseRecord: model.BaseRecord{ - CheckpointID: int64(lsn), - CommitTime: p.commitTime(), - }, + BaseRecord: p.baseRecord(lsn), Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, SourceTableName: tableName, @@ -781,10 +776,7 @@ func (p *PostgresCDCSource) processRelationMessage( // only log audit if there is actionable delta if len(schemaDelta.AddedColumns) > 0 { rec := &model.RelationRecord{ - BaseRecord: model.BaseRecord{ - CheckpointID: int64(lsn), - CommitTime: p.commitTime(), - }, + BaseRecord: p.baseRecord(lsn), TableSchemaDelta: schemaDelta, } return rec, p.auditSchemaDelta(ctx, p.flowJobName, rec) diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go index ba32b27b21..5acbc5962a 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go @@ -48,8 +48,8 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { } rec := &model.InsertRecord{ BaseRecord: model.BaseRecord{ - CheckpointID: 1, - CommitTime: time.Now().UTC(), + CheckpointID: 1, + CommitTimeNano: time.Now().UnixNano(), }, SourceTableName: "test_src_tbl", DestinationTableName: "test_dst_tbl", diff --git a/flow/model/model.go b/flow/model/model.go index 773be38e4c..1561e66544 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -78,8 +78,8 @@ func NewToJSONOptions(unnestCols []string, hstoreAsJSON bool) *ToJSONOptions { type BaseRecord struct { // CheckpointID is the ID of the record. CheckpointID int64 `json:"checkpointId"` - // CommitTime from BeginMessage - CommitTime time.Time `json:"commitTime"` + // BeginMessage.CommitTime.UnixNano(), 16 bytes smaller than time.Time + CommitTimeNano int64 `json:"commitTimeNano"` } func (r *BaseRecord) GetCheckpointID() int64 { @@ -87,7 +87,7 @@ func (r *BaseRecord) GetCheckpointID() int64 { } func (r *BaseRecord) GetCommitTime() time.Time { - return r.CommitTime + return time.Unix(0, r.CommitTimeNano) } type InsertRecord struct { From 521d57631e3a0cda48e4b27cf3bf31e9bf70d641 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 13:46:22 +0000 Subject: [PATCH 25/31] clippy --- nexus/analyzer/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 3213b50ec0..da8cab94d4 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -824,7 +824,6 @@ fn parse_db_options( servers: opts .get("servers") .context("no servers specified")? - .to_string() .split(',') .map(String::from) .collect::>(), From 6bd45b3e21b13b5d62354cdf51ca6e929c6717e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 15:26:56 +0000 Subject: [PATCH 26/31] moved code to gluaflatbuffers / glua64 --- flow/connectors/kafka/kafka.go | 4 +- flow/go.mod | 2 + flow/go.sum | 4 + flow/pua/flatbuffers/LICENSE | 203 --------- flow/pua/flatbuffers/binaryarray.go | 118 ------ flow/pua/flatbuffers/builder.go | 618 ---------------------------- flow/pua/flatbuffers/flatbuffers.go | 38 -- flow/pua/flatbuffers/numtypes.go | 223 ---------- flow/pua/flatbuffers/readme.md | 6 - flow/pua/flatbuffers/view.go | 220 ---------- flow/pua/peerdb.go | 224 ++-------- flow/pua/userdata.go | 41 -- 12 files changed, 31 insertions(+), 1670 deletions(-) delete mode 100644 flow/pua/flatbuffers/LICENSE delete mode 100644 flow/pua/flatbuffers/binaryarray.go delete mode 100644 flow/pua/flatbuffers/builder.go delete mode 100644 flow/pua/flatbuffers/flatbuffers.go delete mode 100644 flow/pua/flatbuffers/numtypes.go delete mode 100644 flow/pua/flatbuffers/readme.md delete mode 100644 flow/pua/flatbuffers/view.go delete mode 100644 flow/pua/userdata.go diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index fe9c397c3b..9fcd7763c8 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -17,13 +17,13 @@ import ( "github.com/yuin/gopher-lua" "go.temporal.io/sdk/log" + "github.com/PeerDB-io/gluaflatbuffers" metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/logger" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/pua" - "github.com/PeerDB-io/peer-flow/pua/flatbuffers" ) type KafkaConnector struct { @@ -196,7 +196,7 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("failed to initialize Lua runtime: %w", err) } } - ls.PreloadModule("flatbuffers", pua_flatbuffers.Loader) + ls.PreloadModule("flatbuffers", gluaflatbuffers.Loader) pua.RegisterTypes(ls) ls.Env.RawSetString("print", ls.NewFunction(func(ls *lua.LState) int { top := ls.GetTop() diff --git a/flow/go.mod b/flow/go.mod index e151b67788..5a5fb83094 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -10,7 +10,9 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.4 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 github.com/ClickHouse/clickhouse-go/v2 v2.21.1 + github.com/PeerDB-io/glua64 v1.0.1 github.com/PeerDB-io/gluabit32 v1.0.0 + github.com/PeerDB-io/gluaflatbuffers v1.0.1 github.com/aws/aws-sdk-go-v2 v1.25.3 github.com/aws/aws-sdk-go-v2/config v1.27.7 github.com/aws/aws-sdk-go-v2/credentials v1.17.7 diff --git a/flow/go.sum b/flow/go.sum index ec8e3499f8..aacfa2da10 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -54,8 +54,12 @@ github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= +github.com/PeerDB-io/glua64 v1.0.1 h1:biXLlFF/L5pnJCwDon7hkWkuQPozC8NjKS3J7Wzi69I= +github.com/PeerDB-io/glua64 v1.0.1/go.mod h1:UHmAhniv61bJPMhQvxkpC7jXbn353dSbQviu83bgQVg= github.com/PeerDB-io/gluabit32 v1.0.0 h1:jn88j22LqiqDoS47LUnvk29hsFQE6yW0imSsHHiYsV8= github.com/PeerDB-io/gluabit32 v1.0.0/go.mod h1:tsHStN1XG5uGVWEA8d/RameB7el3PE3sVkvk8e3+FJg= +github.com/PeerDB-io/gluaflatbuffers v1.0.1 h1:Oxlv0VlMYoQ05Q5n/k4hXAsvtDnuVNC99JBUf927br0= +github.com/PeerDB-io/gluaflatbuffers v1.0.1/go.mod h1:unZOM4Mm2Sn+aAFuVjoJDZ2Dji7jlDWrt4Hvq79as2g= github.com/alecthomas/assert/v2 v2.6.0 h1:o3WJwILtexrEUk3cUVal3oiQY2tfgr/FHWiz/v2n4FU= github.com/alecthomas/assert/v2 v2.6.0/go.mod h1:Bze95FyfUr7x34QZrjL+XP+0qgp/zg8yS+TtBj1WA3k= github.com/alecthomas/repr v0.4.0 h1:GhI2A8MACjfegCPVq9f1FLvIBS+DrQ2KQBFZP1iFzXc= diff --git a/flow/pua/flatbuffers/LICENSE b/flow/pua/flatbuffers/LICENSE deleted file mode 100644 index 6b0b1270ff..0000000000 --- a/flow/pua/flatbuffers/LICENSE +++ /dev/null @@ -1,203 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - diff --git a/flow/pua/flatbuffers/binaryarray.go b/flow/pua/flatbuffers/binaryarray.go deleted file mode 100644 index c51666f0f5..0000000000 --- a/flow/pua/flatbuffers/binaryarray.go +++ /dev/null @@ -1,118 +0,0 @@ -package pua_flatbuffers - -import ( - "github.com/yuin/gopher-lua" - - "github.com/PeerDB-io/peer-flow/pua" -) - -var LuaBinaryArray = pua.UserDataType[[]byte]{Name: "flatbuffers_binaryarray"} - -func BinaryArray_Loader(ls *lua.LState) int { - m := ls.NewTable() - ls.SetField(m, "New", ls.NewFunction(BinaryArrayNew)) - - mt := LuaBinaryArray.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(BinaryArrayIndex)) - ls.SetField(mt, "__len", ls.NewFunction(BinaryArrayLen)) - ls.SetField(mt, "Slice", ls.NewFunction(BinaryArraySlice)) - ls.SetField(mt, "Grow", ls.NewFunction(BinaryArrayGrow)) - ls.SetField(mt, "Pad", ls.NewFunction(BinaryArrayPad)) - ls.SetField(mt, "Set", ls.NewFunction(BinaryArraySet)) - - ls.Push(m) - return 1 -} - -func BinaryArrayNew(ls *lua.LState) int { - lval := ls.Get(1) - var ba []byte - switch val := lval.(type) { - case lua.LString: - ba = []byte(val) - case lua.LNumber: - ba = make([]byte, int(val)) - default: - ls.RaiseError("Expect a integer size value or string to construct a binary array") - return 0 - } - ls.Push(LuaBinaryArray.New(ls, ba)) - return 1 -} - -func BinaryArrayLen(ls *lua.LState) int { - ba := LuaBinaryArray.StartMeta(ls) - ls.Push(lua.LNumber(len(ba))) - return 1 -} - -func BinaryArrayIndex(ls *lua.LState) int { - ba, key := LuaBinaryArray.StartIndex(ls) - switch key { - case "size": - ls.Push(lua.LNumber(len(ba))) - case "str": - ls.Push(lua.LString(ba)) - case "data": - ls.RaiseError("BinaryArray data property inaccessible") - return 0 - default: - ls.Push(ls.GetField(LuaBinaryArray.Metatable(ls), key)) - } - return 1 -} - -func BinaryArraySlice(ls *lua.LState) int { - var startPos, endPos int - ba := LuaBinaryArray.StartMeta(ls) - if luaStartPos, ok := ls.Get(2).(lua.LNumber); ok { - startPos = max(int(luaStartPos), 0) - } else { - startPos = 0 - } - if luaEndPos, ok := ls.Get(3).(lua.LNumber); ok { - endPos = min(int(luaEndPos), len(ba)) - } else { - endPos = len(ba) - } - ls.Push(lua.LString(ba[startPos:endPos])) - return 1 -} - -func BinaryArrayGrow(ls *lua.LState) int { - baud, ba := LuaBinaryArray.Check(ls, 1) - newsize := int(ls.CheckNumber(2)) - if newsize > len(ba) { - newdata := make([]byte, newsize) - copy(newdata[newsize-len(ba):], ba) - baud.Value = newdata - } - return 0 -} - -func Pad(ba []byte, n int, start int) { - for i := range n { - ba[start+i] = 0 - } -} - -func BinaryArrayPad(ls *lua.LState) int { - ba := LuaBinaryArray.StartMeta(ls) - n := int(ls.CheckNumber(2)) - startPos := int(ls.CheckNumber(3)) - Pad(ba, n, startPos) - return 0 -} - -func BinaryArraySet(ls *lua.LState) int { - ba := LuaBinaryArray.StartMeta(ls) - idx := int(ls.CheckNumber(3)) - value := ls.Get(2) - if num, ok := value.(lua.LNumber); ok { - ba[idx] = byte(num) - } - if str, ok := value.(lua.LString); ok { - ba[idx] = str[0] - } - return 0 -} diff --git a/flow/pua/flatbuffers/builder.go b/flow/pua/flatbuffers/builder.go deleted file mode 100644 index 0b4ab18eb8..0000000000 --- a/flow/pua/flatbuffers/builder.go +++ /dev/null @@ -1,618 +0,0 @@ -package pua_flatbuffers - -import ( - "slices" - - "github.com/yuin/gopher-lua" - - "github.com/PeerDB-io/peer-flow/pua" -) - -const VtableMetadataFields int = 2 - -type Builder struct { - ba []byte - vtables []int - currentVT []int - head int - objectEnd int - finished bool - nested bool - minalign uint8 -} - -func (b *Builder) EndVector(ls *lua.LState, vectorSize int) int { - if !b.nested { - ls.RaiseError("EndVector called outside nested context") - return 0 - } - b.nested = false - b.PlaceU64(uint64(vectorSize), uint32n) - return b.Offset() -} - -func (b *Builder) Offset() int { - return len(b.ba) - b.head -} - -func (b *Builder) Pad(pad int) { - if pad > 0 { - b.head -= pad - Pad(b.ba, pad, b.head) - } -} - -func (b *Builder) Place(ls *lua.LState, x lua.LValue, n N) { - b.head -= int(n.width) - n.Pack(ls, b.ba[b.head:], x) -} - -func (b *Builder) PlaceU64(u64 uint64, n N) { - b.head -= int(n.width) - n.PackU64(b.ba[b.head:], u64) -} - -func (b *Builder) Prep(width uint8, additional int) { - if width > b.minalign { - b.minalign = width - } - k := b.Offset() + additional - alignsize := -k & int(width-1) - - space := alignsize + int(width) + additional - - if b.head < space { - oldlen := len(b.ba) - newdata := slices.Grow(b.ba, space) - newdata = newdata[:cap(newdata)] - copy(newdata[len(newdata)-oldlen:], newdata[:oldlen]) - b.head += len(newdata) - oldlen - b.ba = newdata - } - - b.Pad(alignsize) -} - -func (b *Builder) Prepend(ls *lua.LState, n N, x lua.LValue) { - b.Prep(n.width, 0) - b.Place(ls, x, n) -} - -func (b *Builder) PrependU64(n N, x uint64) { - b.Prep(n.width, 0) - b.PlaceU64(x, n) -} - -func (b *Builder) PrependSlot(ls *lua.LState, n N, slotnum int, x lua.LValue, d lua.LValue) { - if !ls.Equal(x, d) { - if xud, ok := x.(*lua.LUserData); ok { - // Need to check int64/number because flatbuffers passes default as 0 - // but Lua only calls __eq when both operands are same type - if dn, ok := d.(lua.LNumber); ok { - switch xv := xud.Value.(type) { - case int64: - if xv == int64(dn) { - return - } - case uint64: - if xv == uint64(dn) { - return - } - } - } - } - - b.Prepend(ls, n, x) - b.Slot(ls, slotnum) - } -} - -func (b *Builder) PrependOffsetTRelative(ls *lua.LState, off int, n N) { - b.Prep(4, 0) - boff := b.Offset() - if off > boff { - ls.RaiseError("Offset arithmetic error") - } else { - b.PlaceU64(uint64(boff-off+4), n) - } -} - -func (b *Builder) PrependSOffsetTRelative(ls *lua.LState, off int) { - b.PrependOffsetTRelative(ls, off, int32n) -} - -func (b *Builder) PrependUOffsetTRelative(ls *lua.LState, off int) { - b.PrependOffsetTRelative(ls, off, uint32n) -} - -func (b *Builder) PrependVOffsetT(off uint16) { - b.Prep(2, 0) - b.PlaceU64(uint64(off), uint16n) -} - -func (b *Builder) Slot(ls *lua.LState, slotnum int) { - if !b.nested { - ls.RaiseError("Slot called outside nested context") - return - } - for slotnum >= len(b.currentVT) { - b.currentVT = append(b.currentVT, 0) - } - b.currentVT[slotnum] = b.Offset() -} - -func vtableEqual(a []int, objectStart int, b []byte) bool { - if len(a)*2 != len(b) { - return false - } - - for i, ai := range a { - x := uint16n.UnpackU64(b[i*2:]) - if (x != 0 || ai != 0) && int(x) != objectStart-ai { - return false - } - } - return true -} - -func (b *Builder) WriteVtable(ls *lua.LState) int { - b.PrependSOffsetTRelative(ls, 0) - objectOffset := b.Offset() - - for len(b.currentVT) > 0 && b.currentVT[len(b.currentVT)-1] == 0 { - b.currentVT = b.currentVT[:len(b.currentVT)-1] - } - - var existingVtable int - for i := len(b.vtables) - 1; i >= 0; i -= 1 { - vt2Offset := b.vtables[i] - vt2Start := len(b.ba) - vt2Offset - vt2Len := uint16n.UnpackU64(b.ba[vt2Start:]) - vt2 := b.ba[vt2Start+VtableMetadataFields*2 : vt2Start+int(vt2Len)] - if vtableEqual(b.currentVT, objectOffset, vt2) { - existingVtable = vt2Offset - break - } - } - - if existingVtable == 0 { - for i := len(b.currentVT) - 1; i >= 0; i -= 1 { - var off uint16 - if b.currentVT[i] != 0 { - off = uint16(objectOffset - b.currentVT[i]) - } - b.PrependVOffsetT(off) - } - - // end each vtable with object size & vtable size - b.PrependVOffsetT(uint16(objectOffset - b.objectEnd)) - b.PrependVOffsetT(uint16(len(b.currentVT)+VtableMetadataFields) * 2) - - newOffset := b.Offset() - int32n.PackU64(b.ba[len(b.ba)-objectOffset:], uint64(newOffset-objectOffset)) - b.vtables = append(b.vtables, newOffset) - } else { - b.head = len(b.ba) - objectOffset - int32n.PackU64(b.ba[b.head:], uint64(existingVtable-objectOffset)) - } - - if len(b.currentVT) != 0 { - b.currentVT = b.currentVT[:0] - } - return objectOffset -} - -var LuaBuilder = pua.UserDataType[*Builder]{Name: "flatbuffers_builder"} - -func Builder_Loader(ls *lua.LState) int { - m := ls.NewTable() - m.RawSetString("New", ls.NewFunction(BuilderNew)) - - mt := LuaBuilder.NewMetatable(ls) - index := ls.SetFuncs(ls.NewTable(), map[string]lua.LGFunction{ - "Clear": BuilderClear, - "Output": BuilderOutput, - "StartObject": BuilderStartObject, - "WriteVtable": BuilderWriteVtable, - "EndObject": BuilderEndObject, - "Head": BuilderHead, - "Offset": BuilderOffset, - "Pad": BuilderPad, - "Prep": BuilderPrep, - "StartVector": BuilderStartVector, - "EndVector": BuilderEndVector, - "CreateString": BuilderCreateString, - "CreateByteVector": BuilderCreateByteVector, - "Slot": BuilderSlot, - "Finish": BuilderFinish, - "FinishSizePrefixed": BuilderFinishSizePrefixed, - "Place": BuilderPlace, - - "PrependSlot": BuilderPrependSlot, - "PrependBoolSlot": BuilderPrependBoolSlot, - "PrependByteSlot": BuilderPrependUint8Slot, - "PrependUint8Slot": BuilderPrependUint8Slot, - "PrependUint16Slot": BuilderPrependUint16Slot, - "PrependUint32Slot": BuilderPrependUint32Slot, - "PrependUint64Slot": BuilderPrependUint64Slot, - "PrependInt8Slot": BuilderPrependInt8Slot, - "PrependInt16Slot": BuilderPrependInt16Slot, - "PrependInt32Slot": BuilderPrependInt32Slot, - "PrependInt64Slot": BuilderPrependInt64Slot, - "PrependFloat32Slot": BuilderPrependFloat32Slot, - "PrependFloat64Slot": BuilderPrependFloat64Slot, - "PrependStructSlot": BuilderPrependStructSlot, - "PrependUOffsetTRelativeSlot": BuilderPrependUOffsetTRelativeSlot, - - "Prepend": BuilderPrepend, - "PrependBool": BuilderPrependBool, - "PrependByte": BuilderPrependUint8, - "PrependUint8": BuilderPrependUint8, - "PrependUint16": BuilderPrependUint16, - "PrependUint32": BuilderPrependUint32, - "PrependUint64": BuilderPrependUint64, - "PrependInt8": BuilderPrependInt8, - "PrependInt16": BuilderPrependInt16, - "PrependInt32": BuilderPrependInt32, - "PrependInt64": BuilderPrependInt64, - "PrependFloat32": BuilderPrependFloat32, - "PrependFloat64": BuilderPrependFloat64, - "PrependVOffsetT": BuilderPrependUint16, - "PrependSOffsetTRelative": BuilderPrependSOffsetTRelative, - "PrependUOffsetTRelative": BuilderPrependUOffsetTRelative, - }) - ls.SetField(mt, "__index", index) - - ls.Push(m) - return 1 -} - -func BuilderNew(ls *lua.LState) int { - initialSize := int(ls.CheckNumber(1)) - - ls.Push(LuaBuilder.New(ls, &Builder{ - ba: make([]byte, initialSize), - vtables: make([]int, 0, 4), - currentVT: make([]int, 0, 4), - head: initialSize, - objectEnd: 0, - finished: false, - nested: false, - minalign: 1, - })) - return 1 -} - -func BuilderClear(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - b.finished = false - b.nested = false - b.minalign = 1 - if len(b.vtables) != 0 { - b.vtables = b.vtables[:0] - } - b.currentVT = b.currentVT[:0] - b.objectEnd = 0 - b.head = len(b.ba) - return 0 -} - -func BuilderOutput(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - if lua.LVIsFalse(ls.Get(2)) { - ls.Push(lua.LString(b.ba[b.head:])) - } else { - ls.Push(lua.LString(b.ba)) - } - return 1 -} - -func BuilderStartObject(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - if b.nested { - ls.RaiseError("StartObject called inside nested context") - return 0 - } - b.nested = true - - numFields := int(ls.CheckNumber(2)) - b.currentVT = slices.Grow(b.currentVT[:0], numFields)[:0] - b.objectEnd = b.Offset() - return 0 -} - -func BuilderWriteVtable(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - b.WriteVtable(ls) - return 0 -} - -func BuilderEndObject(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - if !b.nested { - ls.RaiseError("EndObject called outside nested context") - return 0 - } - b.nested = false - ls.Push(lua.LNumber(b.WriteVtable(ls))) - return 1 -} - -func BuilderHead(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - ls.Push(lua.LNumber(b.head)) - return 1 -} - -func BuilderOffset(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - ls.Push(lua.LNumber(b.Offset())) - return 1 -} - -func BuilderPad(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - pad := ls.CheckNumber(2) - b.Pad(int(pad)) - return 0 -} - -func BuilderPrep(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - size := ls.CheckNumber(2) - additional := ls.CheckNumber(3) - b.Prep(uint8(size), int(additional)) - return 0 -} - -func BuilderPrependSOffsetTRelative(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - b.PrependOffsetTRelative(ls, int(ls.CheckNumber(2)), int32n) - return 0 -} - -func BuilderPrependUOffsetTRelative(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - b.PrependOffsetTRelative(ls, int(ls.CheckNumber(2)), uint32n) - return 0 -} - -func BuilderStartVector(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - if b.nested { - ls.RaiseError("StartVector called in nested context") - } - b.nested = true - elemSize := int(ls.CheckNumber(2)) - numElements := int(ls.CheckNumber(3)) - alignment := uint8(ls.CheckNumber(4)) - elementSize := elemSize * numElements - b.Prep(4, elementSize) - b.Prep(alignment, elementSize) - ls.Push(lua.LNumber(b.Offset())) - return 1 -} - -func BuilderEndVector(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - size := int(ls.CheckNumber(2)) - ls.Push(lua.LNumber(b.EndVector(ls, size))) - return 1 -} - -func createBytesHelper(ls *lua.LState, addnul bool) int { - b := LuaBuilder.StartMeta(ls) - s := ls.CheckString(2) - if b.nested { - if addnul { - ls.RaiseError("CreateString called in nested context") - } else { - ls.RaiseError("CreateByteVector called in nested context") - } - return 0 - } - b.nested = true - - lens := len(s) - if addnul { - b.Prep(4, lens+1) - b.PlaceU64(0, uint8n) - } else { - b.Prep(4, lens) - } - b.head -= lens - copy(b.ba[b.head:], s) - - ls.Push(lua.LNumber(b.EndVector(ls, lens))) - return 1 -} - -func BuilderCreateString(ls *lua.LState) int { - return createBytesHelper(ls, true) -} - -func BuilderCreateByteVector(ls *lua.LState) int { - return createBytesHelper(ls, false) -} - -func BuilderSlot(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - slotnum := int(ls.CheckNumber(2)) - b.Slot(ls, slotnum) - return 0 -} - -func FinishHelper(ls *lua.LState, sizePrefix bool) int { - b := LuaBuilder.StartMeta(ls) - rootTable := int(ls.CheckNumber(2)) - var additional int - if sizePrefix { - additional = 8 - } else { - additional = 4 - } - b.Prep(b.minalign, additional) - b.PrependUOffsetTRelative(ls, rootTable) - if sizePrefix { - b.PrependU64(int32n, uint64(b.Offset())) - } - b.finished = true - ls.Push(lua.LNumber(b.head)) - return 1 -} - -func BuilderFinish(ls *lua.LState) int { - return FinishHelper(ls, false) -} - -func BuilderFinishSizePrefixed(ls *lua.LState) int { - return FinishHelper(ls, true) -} - -func BuilderPlace(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - _, n := LuaN.Check(ls, 3) - b.Place(ls, ls.Get(2), n) - return 0 -} - -func BuilderPrependSlot(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - _, n := LuaN.Check(ls, 2) - slotnum := int(ls.CheckNumber(3)) - b.PrependSlot(ls, n, slotnum, ls.Get(4), ls.Get(5)) - return 0 -} - -func PrependSlotHelper(ls *lua.LState, n N) int { - b := LuaBuilder.StartMeta(ls) - slotnum := int(ls.CheckNumber(2)) - b.PrependSlot(ls, n, slotnum, ls.Get(3), ls.Get(4)) - return 0 -} - -func BuilderPrependBoolSlot(ls *lua.LState) int { - return PrependSlotHelper(ls, booln) -} - -func BuilderPrependUint8Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, uint8n) -} - -func BuilderPrependUint16Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, uint16n) -} - -func BuilderPrependUint32Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, uint32n) -} - -func BuilderPrependUint64Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, uint64n) -} - -func BuilderPrependInt8Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, int8n) -} - -func BuilderPrependInt16Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, int16n) -} - -func BuilderPrependInt32Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, int32n) -} - -func BuilderPrependInt64Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, int64n) -} - -func BuilderPrependFloat32Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, float32n) -} - -func BuilderPrependFloat64Slot(ls *lua.LState) int { - return PrependSlotHelper(ls, float64n) -} - -func BuilderPrependStructSlot(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - x := int(ls.CheckNumber(3)) - d := int(ls.CheckNumber(4)) - if x != d { - if x != b.Offset() { - ls.RaiseError("Tried to write a Struct at an Offset that is different from the current Offset of the Builder.") - } else { - b.Slot(ls, int(ls.CheckNumber(2))) - } - } - return 0 -} - -func BuilderPrependUOffsetTRelativeSlot(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - x := int(ls.CheckNumber(3)) - d := int(ls.CheckNumber(4)) - if x != d { - b.PrependOffsetTRelative(ls, x, uint32n) - b.Slot(ls, int(ls.CheckNumber(2))) - } - return 0 -} - -func BuilderPrepend(ls *lua.LState) int { - b := LuaBuilder.StartMeta(ls) - _, n := LuaN.Check(ls, 2) - b.Prepend(ls, n, ls.Get(3)) - return 0 -} - -func PrependHelper(ls *lua.LState, n N) int { - b := LuaBuilder.StartMeta(ls) - b.Prepend(ls, n, ls.Get(2)) - return 0 -} - -func BuilderPrependBool(ls *lua.LState) int { - return PrependHelper(ls, booln) -} - -func BuilderPrependUint8(ls *lua.LState) int { - return PrependHelper(ls, uint8n) -} - -func BuilderPrependUint16(ls *lua.LState) int { - return PrependHelper(ls, uint16n) -} - -func BuilderPrependUint32(ls *lua.LState) int { - return PrependHelper(ls, uint32n) -} - -func BuilderPrependUint64(ls *lua.LState) int { - return PrependHelper(ls, uint64n) -} - -func BuilderPrependInt8(ls *lua.LState) int { - return PrependHelper(ls, int8n) -} - -func BuilderPrependInt16(ls *lua.LState) int { - return PrependHelper(ls, int16n) -} - -func BuilderPrependInt32(ls *lua.LState) int { - return PrependHelper(ls, int32n) -} - -func BuilderPrependInt64(ls *lua.LState) int { - return PrependHelper(ls, int64n) -} - -func BuilderPrependFloat32(ls *lua.LState) int { - return PrependHelper(ls, float32n) -} - -func BuilderPrependFloat64(ls *lua.LState) int { - return PrependHelper(ls, float64n) -} diff --git a/flow/pua/flatbuffers/flatbuffers.go b/flow/pua/flatbuffers/flatbuffers.go deleted file mode 100644 index 43404b2cda..0000000000 --- a/flow/pua/flatbuffers/flatbuffers.go +++ /dev/null @@ -1,38 +0,0 @@ -package pua_flatbuffers - -import ( - "github.com/yuin/gopher-lua" -) - -func Loader(ls *lua.LState) int { - ls.PreloadModule("flatbuffers.binaryarray", BinaryArray_Loader) - ls.PreloadModule("flatbuffers.builder", Builder_Loader) - ls.PreloadModule("flatbuffers.numTypes", N_Loader) - ls.PreloadModule("flatbuffers.view", View_Loader) - - m := ls.NewTable() - - ls.Push(ls.NewFunction(N_Loader)) - ls.Call(0, 1) - m.RawSetString("N", ls.Get(-1)) - ls.Pop(1) - - ls.Push(ls.NewFunction(View_Loader)) - ls.Call(0, 1) - m.RawSetString("view", ls.Get(-1)) - ls.Pop(1) - - ls.Push(ls.NewFunction(BinaryArray_Loader)) - ls.Call(0, 1) - m.RawSetString("binaryArray", ls.Get(-1)) - ls.Pop(1) - - ls.Push(ls.NewFunction(Builder_Loader)) - ls.Call(0, 1) - builder := ls.GetField(ls.Get(-1), "New") - m.RawSetString("Builder", builder) - ls.Pop(1) - - ls.Push(m) - return 1 -} diff --git a/flow/pua/flatbuffers/numtypes.go b/flow/pua/flatbuffers/numtypes.go deleted file mode 100644 index f3ff6c6f6a..0000000000 --- a/flow/pua/flatbuffers/numtypes.go +++ /dev/null @@ -1,223 +0,0 @@ -package pua_flatbuffers - -import ( - "encoding/binary" - "math" - "strconv" - - "github.com/yuin/gopher-lua" - - "github.com/PeerDB-io/peer-flow/pua" -) - -// Minimal API implemented for generated code - -type Ntype = uint8 - -const ( - tyint Ntype = 0 - tyfloat Ntype = 1 - tybool Ntype = 2 -) - -type N struct { - width uint8 - signed bool - ntype Ntype -} - -var ( - uint8n = N{width: 1, signed: false, ntype: tyint} - uint16n = N{width: 2, signed: false, ntype: tyint} - uint32n = N{width: 4, signed: false, ntype: tyint} - uint64n = N{width: 8, signed: false, ntype: tyint} - int8n = N{width: 1, signed: true, ntype: tyint} - int16n = N{width: 2, signed: true, ntype: tyint} - int32n = N{width: 4, signed: true, ntype: tyint} - int64n = N{width: 8, signed: true, ntype: tyint} - float32n = N{width: 4, signed: true, ntype: tyfloat} - float64n = N{width: 8, signed: true, ntype: tyfloat} - booln = N{width: 1, signed: false, ntype: tybool} -) - -func (n *N) PackU64(buf []byte, val uint64) { - switch n.width { - case 1: - buf[0] = uint8(val) - case 2: - binary.LittleEndian.PutUint16(buf, uint16(val)) - case 4: - binary.LittleEndian.PutUint32(buf, uint32(val)) - case 8: - binary.LittleEndian.PutUint64(buf, val) - default: - panic("Invalid PackU64 width") - } -} - -func (n *N) Pack(ls *lua.LState, buf []byte, val lua.LValue) { - switch n.ntype { - case tyint: - switch lv := val.(type) { - case *lua.LUserData: - switch v := lv.Value.(type) { - case int64: - n.PackU64(buf, uint64(v)) - case uint64: - n.PackU64(buf, v) - default: - n.PackU64(buf, 0) - } - case lua.LNumber: - n.PackU64(buf, uint64(lv)) - case lua.LString: - sv := string(lv) - u64, err := strconv.ParseUint(sv, 10, int(n.width)*8) - if err != nil { - i64, err := strconv.ParseInt(sv, 10, int(n.width)*8) - if err != nil { - n.PackU64(buf, 0) - } else { - n.PackU64(buf, uint64(i64)) - } - } else { - n.PackU64(buf, u64) - } - default: - n.PackU64(buf, 0) - } - case tyfloat: - switch lv := val.(type) { - case *lua.LUserData: - switch v := lv.Value.(type) { - case int64: - n.PackU64(buf, math.Float64bits(float64(v))) - case uint64: - n.PackU64(buf, math.Float64bits(float64(v))) - default: - n.PackU64(buf, 0) - } - case lua.LNumber: - n.PackU64(buf, math.Float64bits(float64(lv))) - case lua.LString: - f64, err := strconv.ParseFloat(string(lv), int(n.width)*8) - if err != nil { - n.PackU64(buf, math.Float64bits(f64)) - } else { - n.PackU64(buf, 0) - } - default: - n.PackU64(buf, 0) - } - case tybool: - if lua.LVIsFalse(val) { - buf[0] = 0 - } else { - buf[1] = 1 - } - } -} - -func (n *N) UnpackU64(buf []byte) uint64 { - switch n.width { - case 1: - return uint64(buf[0]) - case 2: - return uint64(binary.LittleEndian.Uint16(buf)) - case 4: - return uint64(binary.LittleEndian.Uint32(buf)) - case 8: - return binary.LittleEndian.Uint64(buf) - } - panic("invalid bitwidth") -} - -func (n *N) Unpack(ls *lua.LState, buf []byte) lua.LValue { - switch n.ntype { - case tyint: - if !n.signed && n.width < 8 { - return lua.LNumber(n.UnpackU64(buf)) - } - switch n.width { - case 1: - return lua.LNumber(int8(buf[0])) - case 2: - return lua.LNumber(int16(binary.LittleEndian.Uint16(buf))) - case 4: - return lua.LNumber(int32(binary.LittleEndian.Uint32(buf))) - case 8: - u64 := binary.LittleEndian.Uint64(buf) - if n.signed { - return pua.LuaI64.New(ls, int64(u64)) - } else { - return pua.LuaU64.New(ls, u64) - } - } - case tyfloat: - if n.width == 4 { - u32 := binary.LittleEndian.Uint32(buf) - return lua.LNumber(math.Float32frombits(u32)) - } else { - u64 := binary.LittleEndian.Uint64(buf) - return lua.LNumber(math.Float64frombits(u64)) - } - case tybool: - return lua.LBool(buf[0] != 0) - } - panic("invalid numeric metatype") -} - -var LuaN = pua.UserDataType[N]{Name: "flatbuffers_n"} - -func N_Loader(ls *lua.LState) int { - mtidx := ls.CreateTable(0, 1) - mtidx.RawSetString("Unpack", ls.NewFunction(NUnpack)) - mt := LuaView.NewMetatable(ls) - mt.RawSetString("__index", mtidx) - - uint16ud := LuaN.New(ls, uint16n) - uint32ud := LuaN.New(ls, uint32n) - int32ud := LuaN.New(ls, int32n) - - m := ls.NewTable() - m.RawSetString("Uint8", LuaN.New(ls, uint8n)) - m.RawSetString("Uint16", uint16ud) - m.RawSetString("Uint32", uint32ud) - m.RawSetString("Uint64", LuaN.New(ls, uint64n)) - m.RawSetString("Int8", LuaN.New(ls, int8n)) - m.RawSetString("Int16", LuaN.New(ls, int16n)) - m.RawSetString("Int32", int32ud) - m.RawSetString("Int64", LuaN.New(ls, int64n)) - m.RawSetString("Float32", LuaN.New(ls, float32n)) - m.RawSetString("Float64", LuaN.New(ls, float64n)) - m.RawSetString("Bool", LuaN.New(ls, booln)) - - m.RawSetString("UOffsetT", uint32ud) - m.RawSetString("VOffsetT", uint16ud) - m.RawSetString("SOffsetT", int32ud) - - ls.Push(m) - return 1 -} - -func NUnpack(ls *lua.LState) int { - n := LuaN.StartMeta(ls) - pos := max(CheckOffset(ls, 2), 1) - var buf []byte - switch v := ls.Get(1).(type) { - case lua.LString: - buf = []byte(v[pos-1:]) - case *lua.LUserData: - if ba, ok := v.Value.([]byte); ok { - buf = ba[pos-1:] - } else { - ls.RaiseError("Invalid buf userdata passed to unpack") - return 0 - } - default: - ls.RaiseError("Invalid buf passed to unpack") - return 0 - } - ls.Push(n.Unpack(ls, buf)) - return 1 -} diff --git a/flow/pua/flatbuffers/readme.md b/flow/pua/flatbuffers/readme.md deleted file mode 100644 index 494f17e6d3..0000000000 --- a/flow/pua/flatbuffers/readme.md +++ /dev/null @@ -1,6 +0,0 @@ -This directory is licensed under Apache V2 the files are a derivative work of https://github.com/google/flatbuffers/tree/master/lua - -This Go port exists for 3 reasons: -1. upstream lua generator assumes 5.3 (relies on string.unpack/string.pack/integer) despite compatibiltiy shims. gopher-lua is 5.1+goto. So need the code to work with 64 bit integers while Lua 5.1 numbers are all float64 -2. their lua runtime library has some flaws, like encoding bool as ascii "0" or "1" instead of "\0" or "\1" -3. last & definitely least, perf diff --git a/flow/pua/flatbuffers/view.go b/flow/pua/flatbuffers/view.go deleted file mode 100644 index f5b5e698a0..0000000000 --- a/flow/pua/flatbuffers/view.go +++ /dev/null @@ -1,220 +0,0 @@ -package pua_flatbuffers - -import ( - "github.com/yuin/gopher-lua" - - "github.com/PeerDB-io/peer-flow/pua" -) - -type View struct { - ba []byte - pos int // 0-based offset - vtable int // 0-based offset - vtableEnd uint16 - hasv bool -} - -var LuaView = pua.UserDataType[*View]{Name: "flatbuffers_view"} - -func CheckOffset(ls *lua.LState, idx int) int { - num := ls.CheckNumber(idx) - if num < 0 || num > 42949672951 { - ls.RaiseError("Offset is not valid") - } - return int(num) -} - -func View_Loader(ls *lua.LState) int { - m := ls.NewTable() - ls.SetField(m, "New", ls.NewFunction(ViewNew)) - - mt := LuaView.NewMetatable(ls) - ls.SetField(mt, "__index", ls.NewFunction(ViewIndex)) - - ls.Push(m) - return 1 -} - -func ViewNew(ls *lua.LState) int { - buf := ls.Get(1) - var ba []byte - switch val := buf.(type) { - case lua.LString: - ba = []byte(val) - case *lua.LUserData: - var ok bool - ba, ok = val.Value.([]byte) - if !ok { - ls.RaiseError("invalid buf userdata passed to view.New") - return 0 - } - default: - ls.RaiseError("invalid buf passed to view.New") - return 0 - } - ls.Push(LuaView.New(ls, &View{ - ba: ba, - pos: CheckOffset(ls, 2), - })) - return 1 -} - -func ViewIndex(ls *lua.LState) int { - view, key := LuaView.StartIndex(ls) - switch key { - case "bytes": - ls.Push(LuaBinaryArray.New(ls, view.ba)) - case "pos": - ls.Push(lua.LNumber(view.pos)) - case "Offset": - ls.Push(ls.NewFunction(ViewOffset)) - case "Indirect": - ls.Push(ls.NewFunction(ViewIndirect)) - case "String": - ls.Push(ls.NewFunction(ViewString)) - case "VectorLen": - ls.Push(ls.NewFunction(ViewVectorLen)) - case "Vector": - ls.Push(ls.NewFunction(ViewVector)) - case "VectorAsString": - ls.Push(ls.NewFunction(ViewVectorAsString)) - case "Union": - ls.Push(ls.NewFunction(ViewUnion)) - case "Get": - ls.Push(ls.NewFunction(ViewGet)) - case "GetSlot": - ls.Push(ls.NewFunction(ViewGetSlot)) - case "GetVOffsetTSlot": - ls.Push(ls.NewFunction(ViewGetVOffsetTSlot)) - } - return 1 -} - -func (view *View) Offset(vtoff uint16) uint16 { - if !view.hasv { - view.vtable = view.pos - int(int32(int32n.UnpackU64(view.ba[view.pos:]))) - view.vtableEnd = uint16(uint16n.UnpackU64(view.ba[view.vtable:])) - view.hasv = true - } - if vtoff < view.vtableEnd { - return uint16(uint16n.UnpackU64(view.ba[view.vtable+int(vtoff):])) - } else { - return 0 - } -} - -func (view *View) Vector(off int) int { - off += view.pos - return off + int(uint32n.UnpackU64(view.ba[off:])) + 4 -} - -func (view *View) VectorLen(off int) uint32 { - off += int(uint32n.UnpackU64(view.ba[view.pos+off:])) - return uint32(uint32n.UnpackU64(view.ba[off:])) -} - -func ViewOffset(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - vtoff := uint16(CheckOffset(ls, 2)) - ls.Push(lua.LNumber(view.Offset(vtoff))) - return 1 -} - -func ViewIndirect(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - off := CheckOffset(ls, 2) - ls.Push(lua.LNumber(off + int(uint32n.UnpackU64(view.ba[off:])))) - return 1 -} - -func ViewString(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - off := CheckOffset(ls, 2) - off += int(uint32n.UnpackU64(view.ba[off:])) - start := off + 4 - length := int(uint32n.UnpackU64(view.ba[off:])) - ls.Push(lua.LString(view.ba[start : start+length])) - return 1 -} - -func ViewVectorLen(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - off := CheckOffset(ls, 2) - ls.Push(lua.LNumber(view.VectorLen(off))) - return 1 -} - -func ViewVector(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - off := CheckOffset(ls, 2) - ls.Push(lua.LNumber(view.Vector(off))) - return 1 -} - -func ViewVectorAsString(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - off := uint16(CheckOffset(ls, 2)) - o := view.Offset(off) - if o == 0 { - ls.Push(lua.LNil) - return 1 - } - var start, stop int - lstart, ok := ls.Get(3).(lua.LNumber) - if ok { - start = int(lstart) - } - lstop, ok := ls.Get(4).(lua.LNumber) - if ok { - stop = int(lstop) - } else { - stop = int(view.VectorLen(int(o))) - } - a := view.Vector(int(o)) + start - ls.Push(lua.LString(view.ba[a : a+stop-start])) - return 1 -} - -func ViewUnion(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - t2ud, t2 := LuaView.Check(ls, 2) - off := CheckOffset(ls, 3) - off += view.pos - t2.pos = off + int(uint32n.UnpackU64(view.ba[off:])) - t2.ba = view.ba - t2ud.Value = t2 - return 0 -} - -func ViewGet(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - _, n := LuaN.Check(ls, 2) - off := CheckOffset(ls, 3) - ls.Push(n.Unpack(ls, view.ba[off-1:])) - return 1 -} - -func ViewGetSlot(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - slot := uint16(CheckOffset(ls, 2)) - off := view.Offset(slot) - if off == 0 { - ls.Push(ls.Get(3)) - return 1 - } - _, validatorFlags := LuaN.Check(ls, 4) - ls.Push(validatorFlags.Unpack(ls, view.ba[view.pos+int(off):])) - return 1 -} - -func ViewGetVOffsetTSlot(ls *lua.LState) int { - view := LuaView.StartMeta(ls) - slot := uint16(CheckOffset(ls, 2)) - off := view.Offset(slot) - if off == 0 { - ls.Push(ls.Get(3)) - } else { - ls.Push(lua.LNumber(off)) - } - return 1 -} diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 92ef3cefb8..c8f9f8884c 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -4,7 +4,6 @@ import ( "bytes" "fmt" "math/big" - "strconv" "time" "github.com/google/uuid" @@ -12,6 +11,7 @@ import ( "github.com/shopspring/decimal" "github.com/yuin/gopher-lua" + "github.com/PeerDB-io/glua64" "github.com/PeerDB-io/gluabit32" "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/model" @@ -19,17 +19,16 @@ import ( ) var ( - LuaRecord = UserDataType[model.Record]{Name: "peerdb_record"} - LuaRow = UserDataType[*model.RecordItems]{Name: "peerdb_row"} - LuaI64 = UserDataType[int64]{Name: "flatbuffers_i64"} - LuaU64 = UserDataType[uint64]{Name: "flatbuffers_u64"} - LuaTime = UserDataType[time.Time]{Name: "peerdb_time"} - LuaUuid = UserDataType[uuid.UUID]{Name: "peerdb_uuid"} - LuaBigInt = UserDataType[*big.Int]{Name: "peerdb_bigint"} - LuaDecimal = UserDataType[decimal.Decimal]{Name: "peerdb_bigrat"} + LuaRecord = glua64.UserDataType[model.Record]{Name: "peerdb_record"} + LuaRow = glua64.UserDataType[*model.RecordItems]{Name: "peerdb_row"} + LuaTime = glua64.UserDataType[time.Time]{Name: "peerdb_time"} + LuaUuid = glua64.UserDataType[uuid.UUID]{Name: "peerdb_uuid"} + LuaBigInt = glua64.UserDataType[*big.Int]{Name: "peerdb_bigint"} + LuaDecimal = glua64.UserDataType[decimal.Decimal]{Name: "peerdb_bigrat"} ) func RegisterTypes(ls *lua.LState) { + glua64.Loader(ls) ls.Env.RawSetString("loadfile", lua.LNil) ls.Env.RawSetString("dofile", lua.LNil) @@ -51,23 +50,6 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaUuidString)) - eq64 := ls.NewFunction(Lua64Eq) - le64 := ls.NewFunction(Lua64Le) - lt64 := ls.NewFunction(Lua64Lt) - mt = LuaI64.NewMetatable(ls) - mt.RawSetString("__index", ls.NewFunction(LuaI64Index)) - mt.RawSetString("__tostring", ls.NewFunction(LuaI64String)) - mt.RawSetString("__eq", eq64) - mt.RawSetString("__le", le64) - mt.RawSetString("__lt", lt64) - - mt = LuaU64.NewMetatable(ls) - mt.RawSetString("__index", ls.NewFunction(LuaU64Index)) - mt.RawSetString("__tostring", ls.NewFunction(LuaU64String)) - mt.RawSetString("__eq", eq64) - mt.RawSetString("__le", le64) - mt.RawSetString("__lt", lt64) - mt = LuaTime.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaTimeIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaTimeString)) @@ -216,7 +198,7 @@ func LuaRecordIndex(ls *lua.LState) int { ls.Push(lua.LNil) } case "checkpoint": - ls.Push(LuaI64.New(ls, record.GetCheckpointID())) + ls.Push(glua64.I64.New(ls, record.GetCheckpointID())) case "commit_time": ls.Push(LuaTime.New(ls, record.GetCommitTime())) case "target": @@ -254,7 +236,7 @@ func LuaQValue(ls *lua.LState, qv qvalue.QValue) lua.LValue { case int32: return lua.LNumber(v) case int64: - return LuaI64.New(ls, v) + return glua64.I64.New(ls, v) case float32: return lua.LNumber(v) case float64: @@ -293,7 +275,7 @@ func LuaQValue(ls *lua.LState, qv qvalue.QValue) lua.LValue { }) case []int64: return qvToLTable(ls, v, func(x int64) lua.LValue { - return LuaI64.New(ls, x) + return glua64.I64.New(ls, x) }) case []string: return qvToLTable(ls, v, func(x string) lua.LValue { @@ -324,7 +306,7 @@ func LuaUuidIndex(ls *lua.LState) int { } func LuaUuidString(ls *lua.LState) int { - val := LuaUuid.StartMeta(ls) + val := LuaUuid.StartMethod(ls) ls.Push(lua.LString(val.String())) return 1 } @@ -357,177 +339,17 @@ func LuaToString(ls *lua.LState) int { return 0 } -func Lua64Eq(ls *lua.LState) int { - aud := ls.CheckUserData(1) - bud := ls.CheckUserData(2) - switch a := aud.Value.(type) { - case int64: - switch b := bud.Value.(type) { - case int64: - ls.Push(lua.LBool(a == b)) - case uint64: - if a < 0 { - ls.Push(lua.LFalse) - } else { - ls.Push(lua.LBool(uint64(a) == b)) - } - default: - return 0 - } - case uint64: - switch b := bud.Value.(type) { - case int64: - if b < 0 { - ls.Push(lua.LFalse) - } else { - ls.Push(lua.LBool(a == uint64(b))) - } - case uint64: - ls.Push(lua.LBool(a == b)) - default: - return 0 - } - default: - return 0 - } - return 1 -} - -func Lua64Le(ls *lua.LState) int { - aud := ls.CheckUserData(1) - bud := ls.CheckUserData(2) - switch a := aud.Value.(type) { - case int64: - switch b := bud.Value.(type) { - case int64: - ls.Push(lua.LBool(a <= b)) - case uint64: - if a < 0 { - ls.Push(lua.LTrue) - } else { - ls.Push(lua.LBool(uint64(a) <= b)) - } - default: - return 0 - } - case uint64: - switch b := bud.Value.(type) { - case int64: - if b < 0 { - ls.Push(lua.LFalse) - } else { - ls.Push(lua.LBool(a <= uint64(b))) - } - case uint64: - ls.Push(lua.LBool(a <= b)) - default: - return 0 - } - default: - return 0 - } - return 1 -} - -func Lua64Lt(ls *lua.LState) int { - aud := ls.CheckUserData(1) - bud := ls.CheckUserData(2) - switch a := aud.Value.(type) { - case int64: - switch b := bud.Value.(type) { - case int64: - ls.Push(lua.LBool(a < b)) - case uint64: - if a < 0 { - ls.Push(lua.LTrue) - } else { - ls.Push(lua.LBool(uint64(a) < b)) - } - default: - return 0 - } - case uint64: - switch b := bud.Value.(type) { - case int64: - if b < 0 { - ls.Push(lua.LTrue) - } else { - ls.Push(lua.LBool(a < uint64(b))) - } - case uint64: - ls.Push(lua.LBool(a < b)) - default: - return 0 - } - default: - return 0 - } - return 1 -} - -func LuaI64Index(ls *lua.LState) int { - i64ud, i64 := LuaI64.Check(ls, 1) - key := ls.CheckString(2) - switch key { - case "i64": - ls.Push(i64ud) - case "u64": - ls.Push(LuaU64.New(ls, uint64(i64))) - case "float64": - ls.Push(lua.LNumber(i64)) - case "hi": - ls.Push(lua.LNumber(uint32(i64 >> 32))) - case "lo": - ls.Push(lua.LNumber(uint32(i64))) - default: - return 0 - } - return 1 -} - -func LuaU64Index(ls *lua.LState) int { - u64ud, u64 := LuaU64.Check(ls, 1) - key := ls.CheckString(2) - switch key { - case "i64": - ls.Push(LuaI64.New(ls, int64(u64))) - case "u64": - ls.Push(u64ud) - case "float64": - ls.Push(lua.LNumber(u64)) - case "hi": - ls.Push(lua.LNumber(uint32(u64 >> 32))) - case "lo": - ls.Push(lua.LNumber(uint32(u64))) - default: - return 0 - } - return 1 -} - -func LuaI64String(ls *lua.LState) int { - i64 := LuaI64.StartMeta(ls) - ls.Push(lua.LString(strconv.FormatInt(i64, 10))) - return 1 -} - -func LuaU64String(ls *lua.LState) int { - u64 := LuaU64.StartMeta(ls) - ls.Push(lua.LString(strconv.FormatUint(u64, 10))) - return 1 -} - func LuaTimeIndex(ls *lua.LState) int { tm, key := LuaTime.StartIndex(ls) switch key { case "unix_nano": - ls.Push(LuaI64.New(ls, tm.UnixNano())) + ls.Push(glua64.I64.New(ls, tm.UnixNano())) case "unix_micro": - ls.Push(LuaI64.New(ls, tm.UnixMicro())) + ls.Push(glua64.I64.New(ls, tm.UnixMicro())) case "unix_milli": - ls.Push(LuaI64.New(ls, tm.UnixMilli())) + ls.Push(glua64.I64.New(ls, tm.UnixMilli())) case "unix_second": - ls.Push(LuaI64.New(ls, tm.Unix())) + ls.Push(glua64.I64.New(ls, tm.Unix())) case "unix": ls.Push(lua.LNumber(float64(tm.Unix()) + float64(tm.Nanosecond())/1e9)) case "year": @@ -553,7 +375,7 @@ func LuaTimeIndex(ls *lua.LState) int { } func LuaTimeString(ls *lua.LState) int { - tm := LuaTime.StartMeta(ls) + tm := LuaTime.StartMethod(ls) ls.Push(lua.LString(tm.String())) return 1 } @@ -570,7 +392,7 @@ func LuaBigIntIndex(ls *lua.LState) int { case "bytes": ls.Push(lua.LString(bi.Bytes())) case "int64": - ls.Push(LuaI64.New(ls, bi.Int64())) + ls.Push(glua64.I64.New(ls, bi.Int64())) case "is64": ls.Push(lua.LBool(bi.IsInt64())) } @@ -581,13 +403,13 @@ func LuaBigIntIndex(ls *lua.LState) int { } func LuaBigIntString(ls *lua.LState) int { - bi := LuaBigInt.StartMeta(ls) + bi := LuaBigInt.StartMethod(ls) ls.Push(lua.LString(bi.String())) return 1 } func LuaBigIntLen(ls *lua.LState) int { - bi := LuaBigInt.StartMeta(ls) + bi := LuaBigInt.StartMethod(ls) ls.Push(lua.LNumber(len(bi.Bytes()))) return 1 } @@ -598,13 +420,13 @@ func LuaDecimalIndex(ls *lua.LState) int { case "coefficient": ls.Push(LuaBigInt.New(ls, num.Coefficient())) case "coefficient64": - ls.Push(LuaI64.New(ls, num.CoefficientInt64())) + ls.Push(glua64.I64.New(ls, num.CoefficientInt64())) case "exponent": ls.Push(lua.LNumber(num.Exponent())) case "bigint": ls.Push(LuaBigInt.New(ls, num.BigInt())) case "int64": - ls.Push(LuaI64.New(ls, num.IntPart())) + ls.Push(glua64.I64.New(ls, num.IntPart())) case "float64": ls.Push(lua.LNumber(num.InexactFloat64())) default: @@ -614,7 +436,7 @@ func LuaDecimalIndex(ls *lua.LState) int { } func LuaDecimalString(ls *lua.LState) int { - num := LuaDecimal.StartMeta(ls) + num := LuaDecimal.StartMethod(ls) ls.Push(lua.LString(num.String())) return 1 } diff --git a/flow/pua/userdata.go b/flow/pua/userdata.go deleted file mode 100644 index e0dba8254f..0000000000 --- a/flow/pua/userdata.go +++ /dev/null @@ -1,41 +0,0 @@ -package pua - -import ( - "github.com/yuin/gopher-lua" -) - -type UserDataType[T any] struct{ Name string } - -func (udt *UserDataType[T]) New(ls *lua.LState, val T) *lua.LUserData { - return &lua.LUserData{ - Value: val, - Env: ls.Env, - Metatable: udt.Metatable(ls), - } -} - -func (udt *UserDataType[T]) NewMetatable(ls *lua.LState) *lua.LTable { - return ls.NewTypeMetatable(udt.Name) -} - -func (udt *UserDataType[T]) Metatable(ls *lua.LState) lua.LValue { - return ls.GetTypeMetatable(udt.Name) -} - -func (udt *UserDataType[T]) Check(ls *lua.LState, idx int) (*lua.LUserData, T) { - ud := ls.CheckUserData(idx) - val, ok := ud.Value.(T) - if !ok { - ls.RaiseError("Invalid " + udt.Name) - } - return ud, val -} - -func (udt *UserDataType[T]) StartMeta(ls *lua.LState) T { - _, val := udt.Check(ls, 1) - return val -} - -func (udt *UserDataType[T]) StartIndex(ls *lua.LState) (T, string) { - return udt.StartMeta(ls), ls.CheckString(2) -} From 49200e5ce56c70f80c7b5501f222ecd88f6d86a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 17:00:45 +0000 Subject: [PATCH 27/31] kafka: split up SyncRecords, report numrecords as records consumed, not produced --- flow/connectors/kafka/kafka.go | 184 +++++++++++++++++---------------- 1 file changed, 96 insertions(+), 88 deletions(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 9fcd7763c8..636f173d98 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -158,27 +158,11 @@ func (c *KafkaConnector) SyncFlowCleanup(ctx context.Context, jobName string) er return c.pgMetadata.DropMetadata(ctx, jobName) } -func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - // TODO BeginTransaction if transactional - - var wg sync.WaitGroup - wgCtx, wgErr := context.WithCancelCause(ctx) - produceCb := func(r *kgo.Record, err error) { - if err != nil { - wgErr(err) - } - wg.Done() - } - - numRecords := int64(0) - tableNameRowsMapping := make(map[string]uint32) - - var fn *lua.LFunction - var ls *lua.LState - if req.Script != "" { - ls = lua.NewState(lua.Options{SkipOpenLibs: true}) +func loadScript(ctx context.Context, script string, print lua.LGFunction) (*lua.LState, error) { + if script != "" { + ls := lua.NewState(lua.Options{SkipOpenLibs: true}) defer ls.Close() - ls.SetContext(wgCtx) + ls.SetContext(ctx) for _, pair := range []struct { n string f lua.LGFunction @@ -198,33 +182,100 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords } ls.PreloadModule("flatbuffers", gluaflatbuffers.Loader) pua.RegisterTypes(ls) - ls.Env.RawSetString("print", ls.NewFunction(func(ls *lua.LState) int { - top := ls.GetTop() - ss := make([]string, top) - for i := range top { - ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() - } - _ = c.pgMetadata.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) - return 0 - })) - err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(req.Script)) + ls.Env.RawSetString("print", ls.NewFunction(print)) + err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(script)) if err != nil { - return nil, fmt.Errorf("error loading script %s: %w", req.Script, err) + return nil, fmt.Errorf("error loading script %s: %w", script, err) } err = ls.PCall(0, 0, nil) if err != nil { - return nil, fmt.Errorf("error executing script %s: %w", req.Script, err) - } - - var ok bool - lfn := ls.Env.RawGetString("onRecord") - fn, ok = lfn.(*lua.LFunction) - if !ok { - return nil, fmt.Errorf("script should define `onRecord` as function, not %s", lfn) + return nil, fmt.Errorf("error executing script %s: %w", script, err) } + return ls, nil } else { return nil, errors.New("kafka mirror must have script") } +} + +func lvalueToKafkaRecord(ls *lua.LState, value lua.LValue) (*kgo.Record, error) { + var kr *kgo.Record + switch v := value.(type) { + case lua.LString: + kr = kgo.StringRecord(string(v)) + case *lua.LTable: + key, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "key")) + if err != nil { + return nil, fmt.Errorf("invalid key, %w", err) + } + value, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "value")) + if err != nil { + return nil, fmt.Errorf("invalid value, %w", err) + } + topic, err := LVAsStringOrNil(ls, ls.GetField(v, "topic")) + if err != nil { + return nil, fmt.Errorf("invalid topic, %w", err) + } + partition := int32(lua.LVAsNumber(ls.GetField(v, "partition"))) + kr = &kgo.Record{ + Key: key, + Value: value, + Topic: topic, + Partition: partition, + } + lheaders := ls.GetField(v, "headers") + if headers, ok := lheaders.(*lua.LTable); ok { + headers.ForEach(func(k, v lua.LValue) { + kstr := k.String() + vbytes, err := LVAsReadOnlyBytes(ls, v) + if err != nil { + vbytes = unsafeFastStringToReadOnlyBytes(err.Error()) + } + kr.Headers = append(kr.Headers, kgo.RecordHeader{ + Key: kstr, + Value: vbytes, + }) + }) + } else if lua.LVAsBool(lheaders) { + return nil, fmt.Errorf("invalid headers, must be nil or table: %s", lheaders) + } + case *lua.LNilType: + default: + return nil, fmt.Errorf("script returned invalid value: %s", value) + } + return kr, nil +} + +func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { + var wg sync.WaitGroup + wgCtx, wgErr := context.WithCancelCause(ctx) + produceCb := func(r *kgo.Record, err error) { + if err != nil { + wgErr(err) + } + wg.Done() + } + + numRecords := int64(0) + tableNameRowsMapping := make(map[string]uint32) + + ls, err := loadScript(wgCtx, req.Script, func(ls *lua.LState) int { + top := ls.GetTop() + ss := make([]string, top) + for i := range top { + ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + } + _ = c.pgMetadata.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) + return 0 + }) + if err != nil { + return nil, err + } + + lfn := ls.Env.RawGetString("onRecord") + fn, ok := lfn.(*lua.LFunction) + if !ok { + return nil, fmt.Errorf("script should define `onRecord` as function, not %s", lfn) + } for record := range req.Records.GetRecords() { if err := wgCtx.Err(); err != nil { @@ -238,50 +289,9 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords } args := ls.GetTop() for i := range args { - value := ls.Get(i - args) - var kr *kgo.Record - switch v := value.(type) { - case lua.LString: - kr = kgo.StringRecord(string(v)) - case *lua.LTable: - key, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "key")) - if err != nil { - return nil, fmt.Errorf("invalid key, %w", err) - } - value, err := LVAsReadOnlyBytes(ls, ls.GetField(v, "value")) - if err != nil { - return nil, fmt.Errorf("invalid value, %w", err) - } - topic, err := LVAsStringOrNil(ls, ls.GetField(v, "topic")) - if err != nil { - return nil, fmt.Errorf("invalid topic, %w", err) - } - partition := int32(lua.LVAsNumber(ls.GetField(v, "partition"))) - kr = &kgo.Record{ - Key: key, - Value: value, - Topic: topic, - Partition: partition, - } - lheaders := ls.GetField(v, "headers") - if headers, ok := lheaders.(*lua.LTable); ok { - headers.ForEach(func(k, v lua.LValue) { - kstr := k.String() - vbytes, err := LVAsReadOnlyBytes(ls, v) - if err != nil { - vbytes = unsafeFastStringToReadOnlyBytes(err.Error()) - } - kr.Headers = append(kr.Headers, kgo.RecordHeader{ - Key: kstr, - Value: vbytes, - }) - }) - } else if lua.LVAsBool(lheaders) { - return nil, fmt.Errorf("invalid headers, must be nil or table: %s", lheaders) - } - case *lua.LNilType: - default: - return nil, fmt.Errorf("script returned invalid value: %s", value) + kr, err := lvalueToKafkaRecord(ls, ls.Get(i-args)) + if err != nil { + return nil, err } if kr != nil { if kr.Topic == "" { @@ -290,17 +300,17 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords wg.Add(1) c.client.Produce(wgCtx, kr, produceCb) - numRecords += 1 tableNameRowsMapping[kr.Topic] += 1 } } + numRecords += 1 ls.SetTop(0) } waitChan := make(chan struct{}) go func() { wg.Wait() - waitChan <- struct{}{} + close(waitChan) }() select { case <-wgCtx.Done(): @@ -312,10 +322,8 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords return nil, fmt.Errorf("could not flush transaction: %w", err) } - // TODO EndTransaction if transactional - lastCheckpoint := req.Records.GetLastCheckpoint() - err := c.pgMetadata.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) + err = c.pgMetadata.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) if err != nil { return nil, err } From cb968208eaf1b7bcc4a824b22ca5aeb15b0dea04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 18:45:29 +0000 Subject: [PATCH 28/31] loadScript cleanup control flow --- flow/connectors/kafka/kafka.go | 66 +++++++++++++++++----------------- 1 file changed, 33 insertions(+), 33 deletions(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 636f173d98..2090c5e617 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -159,42 +159,42 @@ func (c *KafkaConnector) SyncFlowCleanup(ctx context.Context, jobName string) er } func loadScript(ctx context.Context, script string, print lua.LGFunction) (*lua.LState, error) { - if script != "" { - ls := lua.NewState(lua.Options{SkipOpenLibs: true}) - defer ls.Close() - ls.SetContext(ctx) - for _, pair := range []struct { - n string - f lua.LGFunction - }{ - {lua.LoadLibName, lua.OpenPackage}, // Must be first - {lua.BaseLibName, lua.OpenBase}, - {lua.TabLibName, lua.OpenTable}, - {lua.StringLibName, lua.OpenString}, - {lua.MathLibName, lua.OpenMath}, - } { - ls.Push(ls.NewFunction(pair.f)) - ls.Push(lua.LString(pair.n)) - err := ls.PCall(1, 0, nil) - if err != nil { - return nil, fmt.Errorf("failed to initialize Lua runtime: %w", err) - } - } - ls.PreloadModule("flatbuffers", gluaflatbuffers.Loader) - pua.RegisterTypes(ls) - ls.Env.RawSetString("print", ls.NewFunction(print)) - err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(script)) - if err != nil { - return nil, fmt.Errorf("error loading script %s: %w", script, err) - } - err = ls.PCall(0, 0, nil) + if script == "" { + return nil, errors.New("kafka mirror must have script") + } + + ls := lua.NewState(lua.Options{SkipOpenLibs: true}) + defer ls.Close() + ls.SetContext(ctx) + for _, pair := range []struct { + n string + f lua.LGFunction + }{ + {lua.LoadLibName, lua.OpenPackage}, // Must be first + {lua.BaseLibName, lua.OpenBase}, + {lua.TabLibName, lua.OpenTable}, + {lua.StringLibName, lua.OpenString}, + {lua.MathLibName, lua.OpenMath}, + } { + ls.Push(ls.NewFunction(pair.f)) + ls.Push(lua.LString(pair.n)) + err := ls.PCall(1, 0, nil) if err != nil { - return nil, fmt.Errorf("error executing script %s: %w", script, err) + return nil, fmt.Errorf("failed to initialize Lua runtime: %w", err) } - return ls, nil - } else { - return nil, errors.New("kafka mirror must have script") } + ls.PreloadModule("flatbuffers", gluaflatbuffers.Loader) + pua.RegisterTypes(ls) + ls.Env.RawSetString("print", ls.NewFunction(print)) + err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(script)) + if err != nil { + return nil, fmt.Errorf("error loading script %s: %w", script, err) + } + err = ls.PCall(0, 0, nil) + if err != nil { + return nil, fmt.Errorf("error executing script %s: %w", script, err) + } + return ls, nil } func lvalueToKafkaRecord(ls *lua.LState, value lua.LValue) (*kgo.Record, error) { From 3282859d0d112f58a86395da0551bc845f23b7cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 19:00:18 +0000 Subject: [PATCH 29/31] move glua64 tests to glua64 --- flow/pua/peerdb_test.go | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/flow/pua/peerdb_test.go b/flow/pua/peerdb_test.go index fdbcd53295..b2a752a16b 100644 --- a/flow/pua/peerdb_test.go +++ b/flow/pua/peerdb_test.go @@ -25,14 +25,6 @@ func Test_Lua(t *testing.T) { id := uuid.UUID([16]byte{2, 3, 5, 7, 11, 13, 17, 19, 127, 131, 137, 139, 149, 151, 241, 251}) ls.Env.RawSetString("uuid", LuaUuid.New(ls, id)) - n5 := int64(-5) - ls.Env.RawSetString("i64p5", LuaI64.New(ls, 5)) - ls.Env.RawSetString("i64p5_2", LuaI64.New(ls, 5)) - ls.Env.RawSetString("u64p5", LuaU64.New(ls, 5)) - ls.Env.RawSetString("u64p5_2", LuaU64.New(ls, 5)) - ls.Env.RawSetString("i64n5", LuaI64.New(ls, n5)) - ls.Env.RawSetString("u64n5", LuaU64.New(ls, uint64(n5))) - assert(t, ls, ` assert(require('bit32').band(173, 21) == 5) assert(dofile == nil) @@ -54,14 +46,5 @@ assert(uuid[12] == 149) assert(uuid[13] == 151) assert(uuid[14] == 241) assert(uuid[15] == 251) - -print(i64p5, u64p5) -assert(i64p5 == u64p5) -assert(i64p5 ~= i64n5) -assert(i64n5 ~= u64n5) -assert(i64p5 == i64p5_2) -assert(u64p5 == u64p5_2) -assert(u64n5 > i64p5) -assert(i64p5 > i64n5) `) } From 19314c0ceef1dd29db42a59f11ef3c1127b122af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 19:07:05 +0000 Subject: [PATCH 30/31] fix lint --- flow/connectors/kafka/kafka.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 2090c5e617..ff491ddc8c 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -158,7 +158,7 @@ func (c *KafkaConnector) SyncFlowCleanup(ctx context.Context, jobName string) er return c.pgMetadata.DropMetadata(ctx, jobName) } -func loadScript(ctx context.Context, script string, print lua.LGFunction) (*lua.LState, error) { +func loadScript(ctx context.Context, script string, printfn lua.LGFunction) (*lua.LState, error) { if script == "" { return nil, errors.New("kafka mirror must have script") } @@ -185,7 +185,7 @@ func loadScript(ctx context.Context, script string, print lua.LGFunction) (*lua. } ls.PreloadModule("flatbuffers", gluaflatbuffers.Loader) pua.RegisterTypes(ls) - ls.Env.RawSetString("print", ls.NewFunction(print)) + ls.Env.RawSetString("print", ls.NewFunction(printfn)) err := ls.GPCall(pua.LoadPeerdbScript, lua.LString(script)) if err != nil { return nil, fmt.Errorf("error loading script %s: %w", script, err) From 0a81adea2297c7988f180d38536a9f5326ddccd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Mar 2024 20:32:40 +0000 Subject: [PATCH 31/31] oops --- flow/connectors/kafka/kafka.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index ff491ddc8c..b879e31def 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -164,7 +164,6 @@ func loadScript(ctx context.Context, script string, printfn lua.LGFunction) (*lu } ls := lua.NewState(lua.Options{SkipOpenLibs: true}) - defer ls.Close() ls.SetContext(ctx) for _, pair := range []struct { n string @@ -270,6 +269,7 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords if err != nil { return nil, err } + defer ls.Close() lfn := ls.Env.RawGetString("onRecord") fn, ok := lfn.(*lua.LFunction)