diff --git a/materialize-bigquery/.snapshots/TestValidateAndApply b/materialize-bigquery/.snapshots/TestValidateAndApply index 094a653bf2..5e0d00cf13 100644 --- a/materialize-bigquery/.snapshots/TestValidateAndApply +++ b/materialize-bigquery/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BOOLEAN' but endpoint type 'INTEGER' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'INTEGER' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'STRING' but endpoint type 'INTEGER' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'BIGNUMERIC' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'BIGNUMERIC' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'BIGNUMERIC' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-bigquery/bigquery.go b/materialize-bigquery/bigquery.go index fd8e3e4aee..bab0fc01f2 100644 --- a/materialize-bigquery/bigquery.go +++ b/materialize-bigquery/bigquery.go @@ -154,14 +154,10 @@ func newBigQueryDriver() *sql.Driver { "bucket_path": cfg.BucketPath, }).Info("creating bigquery endpoint") - var metaBase sql.TablePath = []string{cfg.ProjectID, cfg.Dataset} - var metaSpecs, metaCheckpoints = sql.MetaTables(metaBase) - return &sql.Endpoint{ Config: cfg, Dialect: bqDialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable([]string{cfg.ProjectID, cfg.Dataset}), NewClient: newClient, CreateTableTemplate: tplCreateTargetTable, NewResource: newTableConfig, diff --git a/materialize-bigquery/bigquery_test.go b/materialize-bigquery/bigquery_test.go index 8eb0a13514..8ea7824c6f 100644 --- a/materialize-bigquery/bigquery_test.go +++ b/materialize-bigquery/bigquery_test.go @@ -14,7 +14,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/google/uuid" "github.com/stretchr/testify/require" @@ -73,18 +72,12 @@ func TestValidateAndApply(t *testing.T) { t.Helper() return dumpSchema(t, ctx, client, cfg, resourceConfig) }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = client.query(ctx, fmt.Sprintf( "drop table %s;", bqDialect.Identifier(cfg.ProjectID, cfg.Dataset, resourceConfig.Table), )) - - _, _ = client.query(ctx, fmt.Sprintf( - "delete from %s where materialization = 'test/sqlite'", - bqDialect.Identifier(cfg.ProjectID, cfg.Dataset, sql.DefaultFlowMaterializations), - )) }, ) } @@ -129,7 +122,7 @@ func TestValidateAndApplyMigrations(t *testing.T) { // bigquery does not support more than 6 fractional second precision, and will fail if we try // to insert a value with 9 - for i, _ := range values { + for i := range values { if keys[i] == "datetimeValue" { values[i] = "'2024-01-01 01:01:01.111111'" } @@ -186,18 +179,12 @@ func TestValidateAndApplyMigrations(t *testing.T) { return b.String() }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = client.query(ctx, fmt.Sprintf( "drop table %s;", bqDialect.Identifier(cfg.ProjectID, cfg.Dataset, resourceConfig.Table), )) - - _, _ = client.query(ctx, fmt.Sprintf( - "delete from %s where materialization = 'test/sqlite'", - bqDialect.Identifier(cfg.ProjectID, cfg.Dataset, sql.DefaultFlowMaterializations), - )) }, ) } @@ -287,10 +274,6 @@ func TestFencingCases(t *testing.T) { } func TestPrereqs(t *testing.T) { - // These tests assume that the configuration obtained from environment variables forms a valid - // config that could be used to materialize into Bigquery. Various parameters of the - // configuration are then manipulated to test assertions for incorrect configs. - // Due to the nature of configuring the connector with a JSON service account key and the // difficulties in discriminating between error responses from BigQuery there's only a handful // of cases that can be explicitly tested. diff --git a/materialize-bigquery/client.go b/materialize-bigquery/client.go index a9808830e5..405acee548 100644 --- a/materialize-bigquery/client.go +++ b/materialize-bigquery/client.go @@ -2,7 +2,6 @@ package connector import ( "context" - dbSql "database/sql" "encoding/base64" "errors" "fmt" @@ -16,7 +15,6 @@ import ( storage "cloud.google.com/go/storage" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" "github.com/google/uuid" log "github.com/sirupsen/logrus" "golang.org/x/sync/errgroup" @@ -113,11 +111,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return is, nil } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.query(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { _, err := c.query(ctx, tc.TableCreateSql) return err @@ -316,36 +309,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (specB64, version string, err error) { - job, err := c.query(ctx, fmt.Sprintf( - "SELECT version, spec FROM %s WHERE materialization=%s;", - specs.Identifier, - specs.Keys[0].Placeholder, - ), materialization.String()) - if err != nil { - return "", "", err - } - - var data struct { - Version string `bigquery:"version"` - SpecB64 string `bigquery:"spec"` - } - - if err := c.fetchOne(ctx, job, &data); err == errNotFound { - return "", "", dbSql.ErrNoRows - } else if err != nil { - return "", "", err - } - - log.WithFields(log.Fields{ - "table": specs.Identifier, - "materialization": materialization.String(), - "version": data.Version, - }).Info("existing materialization spec loaded") - - return data.SpecB64, data.Version, nil -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { _, err := c.query(ctx, strings.Join(statements, "\n")) return err diff --git a/materialize-bigquery/query.go b/materialize-bigquery/query.go index 45308a3caa..de88514a0f 100644 --- a/materialize-bigquery/query.go +++ b/materialize-bigquery/query.go @@ -50,11 +50,29 @@ func (c client) runQuery(ctx context.Context, query *bigquery.Query) (*bigquery. var job *bigquery.Job var err error for attempt := 0; attempt < maxAttempts; attempt++ { + backoff *= math.Pow(2, 1+rand.Float64()) + if backoff > maxBackoff { + backoff = maxBackoff + } + retryDelay := time.Duration(backoff * float64(time.Millisecond)) + job, err = query.Run(ctx) if err != nil { return nil, fmt.Errorf("run: %w", err) } + // Some queries may immediately fail, such as table alteration rate + // limits. For these, `job.Wait` will hang forever, so we must check to + // see if the job has already failed. + if initialStatus, err := job.Status(ctx); err != nil { + return nil, fmt.Errorf("getting initialStatus for job: %w", err) + } else if err := initialStatus.Err(); err != nil { + if err := maybeRetry(ctx, err, attempt, retryDelay, initialStatus); err != nil { + return nil, err + } + continue + } + // Weirdness ahead: if `err != nil`, then `status` might be nil. But if `err == nil`, then // there might still have been an error reported by `status.Err()`. We always want both the // err and the status so that we can check both. When `err != nil`, the status may still @@ -67,61 +85,11 @@ func (c client) runQuery(ctx context.Context, query *bigquery.Query) (*bigquery. if err == nil { err = status.Err() } - if err != nil { - // Is this a terminal error? - - // We need to retry errors due to concurrent updates to the same table, but - // unfortunately there's no good way to identify such errors. The status code of that - // error is 400 and the status is "invalidQuery" (see - // https://cloud.google.com/bigquery/docs/error-messages), which also applies to several - // other scenarios like the instance being fenced off (from our use of RAISE), a table - // being referenced by the query not existing (which is for some reason not a 404), etc. - - // Because of this we match on substrings in the error message to determine if a retry - // should be attempted. The two errors that may be encountered from concurrent shards - // operating in the same dataset have the error strings "Transaction is aborted due to - // concurrent update against table ..." (most common, seemingly), or "Could not - // serialize access to table ...". We retry only if the the error string contains these - // strings. - - // Short term rate limit errors can also be retried using the same exponential backoff - // strategy. These kinds of errors can always be identified by their "Reason" being - // "jobRateLimitExceeded". - if e, ok := err.(*googleapi.Error); ok { - if strings.Contains(err.Error(), "Transaction is aborted due to concurrent update against table") || - strings.Contains(err.Error(), "Could not serialize access to table") || - strings.Contains(err.Error(), "The job encountered an error during execution. Retrying the job may solve the problem.") || - (len(e.Errors) == 1 && e.Errors[0].Reason == "jobRateLimitExceeded") { - backoff *= math.Pow(2, 1+rand.Float64()) - if backoff > maxBackoff { - backoff = maxBackoff - } - delay := time.Duration(backoff * float64(time.Millisecond)) - - ll := log.WithFields(log.Fields{ - "attempt": attempt, - "jobStatus": status, - "error": err, - "delay": delay.String(), - }) - - if attempt > 10 { - ll.Info("job failed (will retry)") - } else { - ll.Debug("job failed (will retry)") - } - - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-time.After(delay): - continue - } - } + if err := maybeRetry(ctx, err, attempt, retryDelay, status); err != nil { + return nil, err } - - return nil, err + continue } // I think this is just documenting the assumption that the job must always be Done after @@ -136,6 +104,70 @@ func (c client) runQuery(ctx context.Context, query *bigquery.Query) (*bigquery. return job, fmt.Errorf("exhausted retries: %w", err) } +// maybeRetry will return a `nil` error after a period of time if the provided +// `err` is retryable. Otherwise it will return the provided error immediately. +func maybeRetry(ctx context.Context, err error, attempt int, delay time.Duration, status *bigquery.JobStatus) error { + doDelay := func() error { + ll := log.WithFields(log.Fields{ + "attempt": attempt, + "jobStatus": status, + "error": err, + "delay": delay.String(), + }) + + if attempt > 10 { + ll.Info("job failed (will retry)") + } else { + ll.Debug("job failed (will retry)") + } + + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(delay): + return nil + } + } + + // We need to retry errors due to concurrent updates to the same table, but + // unfortunately there's no good way to identify such errors. The status code of that + // error is 400 and the status is "invalidQuery" (see + // https://cloud.google.com/bigquery/docs/error-messages), which also applies to several + // other scenarios like the instance being fenced off (from our use of RAISE), a table + // being referenced by the query not existing (which is for some reason not a 404), etc. + + // Because of this we match on substrings in the error message to determine if a retry + // should be attempted. The two errors that may be encountered from concurrent shards + // operating in the same dataset have the error strings "Transaction is aborted due to + // concurrent update against table ..." (most common, seemingly), or "Could not + // serialize access to table ...". We retry only if the the error string contains these + // strings. + + // Short term rate limit errors can also be retried using the same exponential backoff + // strategy. These kinds of errors can always be identified by their "Reason" being + // "jobRateLimitExceeded". + if e, ok := err.(*googleapi.Error); ok { + if strings.Contains(err.Error(), "Transaction is aborted due to concurrent update against table") || + strings.Contains(err.Error(), "Could not serialize access to table") || + strings.Contains(err.Error(), "The job encountered an error during execution. Retrying the job may solve the problem.") || + (len(e.Errors) == 1 && e.Errors[0].Reason == "jobRateLimitExceeded") { + return doDelay() + } + } + + // A *bigquery.Error is returned if the job immediately has an error in its + // status. The only way I have found this to happen from our usage is from + // table alterations exceeding a rate limit. + if e, ok := err.(*bigquery.Error); ok { + if e.Reason == "rateLimitExceeded" { + return doDelay() + } + } + + // Not a retryable error. + return err +} + // fetchOne will fetch one row of data from a job and scan it into dest. func (c client) fetchOne(ctx context.Context, job *bigquery.Job, dest interface{}) error { diff --git a/materialize-boilerplate/.snapshots/TestApply b/materialize-boilerplate/.snapshots/TestApply index 43c6c8c7ff..60e6adfb0e 100644 --- a/materialize-boilerplate/.snapshots/TestApply +++ b/materialize-boilerplate/.snapshots/TestApply @@ -1,30 +1,18 @@ * new materialization: -create meta tables -create resource for collection "key/value" -put spec with version "aVersion" +create resource for ["key_value"] * remove required field: -create meta tables -update resource for collection "key/value" [new projections: 0, newly nullable fields: 1, newly delta updates: false] -put spec with version "aVersion" +update resource for ["key_value"] [new projections: 0, newly nullable fields: 1, newly delta updates: false] * add required field: -create meta tables -update resource for collection "key/value" [new projections: 1, newly nullable fields: 0, newly delta updates: false] -put spec with version "aVersion" +update resource for ["key_value"] [new projections: 1, newly nullable fields: 0, newly delta updates: false] * add binding: -create meta tables -create resource for collection "extra/collection" -put spec with version "aVersion" +create resource for ["extra_collection"] * replace binding: -create meta tables delete resource ["key_value"] -create resource for collection "key/value" -put spec with version "aVersion" +create resource for ["key_value"] * field is newly nullable: -create meta tables -update resource for collection "key/value" [new projections: 0, newly nullable fields: 1, newly delta updates: false] -put spec with version "aVersion" +update resource for ["key_value"] [new projections: 0, newly nullable fields: 1, newly delta updates: false] diff --git a/materialize-boilerplate/apply.go b/materialize-boilerplate/apply.go index 5e4815f0f0..c20e82ad66 100644 --- a/materialize-boilerplate/apply.go +++ b/materialize-boilerplate/apply.go @@ -43,16 +43,6 @@ type BindingUpdate struct { // resources based on binding changes. Many of these functions should return an ActionApplyFn, which // may be executed concurrently. type Applier interface { - // CreateMetaTables is called to create the tables (or the equivalent endpoint concept) that - // store a persisted spec and any other metadata the materialization needs to persist. - CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, ActionApplyFn, error) - - // LoadSpec loads the persisted spec from the metadata table. - LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) - - // PutSpec upserts a spec into the metadata table. - PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, exists bool) (string, ActionApplyFn, error) - // CreateResource creates a new resource in the endpoint. It is called only if the resource does // not already exist, either because it is brand new or because it was previously deleted as // part of a resource replacement. @@ -82,11 +72,6 @@ func ApplyChanges(ctx context.Context, req *pm.Request_Apply, applier Applier, i return nil, fmt.Errorf("validating request: %w", err) } - storedSpec, err := applier.LoadSpec(ctx, req.Materialization.Name) - if err != nil { - return nil, fmt.Errorf("getting stored spec: %w", err) - } - actionDescriptions := []string{} actions := []ActionApplyFn{} @@ -97,22 +82,11 @@ func ApplyChanges(ctx context.Context, req *pm.Request_Apply, applier Applier, i } } - // TODO(whb): We will eventually stop persisting specs for materializations, and instead include - // the previous spec as part of the protocol. When that happens this can go away. Then, if - // individual materializations still need individual metadata tables (ex: SQL materializations), - // they should create them as needed separately from the Applier. For now, we always call - // CreateMetaTables, and materializations are free to either create them or not. - desc, action, err := applier.CreateMetaTables(ctx, req.Materialization) - if err != nil { - return nil, fmt.Errorf("getting CreateMetaTables action: %w", err) - } - addAction(desc, action) - for bindingIdx, binding := range req.Materialization.Bindings { // The existing binding spec is used to extract various properties that can't be learned // from introspecting the destination system, such as the backfill counter and if the // materialization was previously delta updates. - existingBinding, err := findExistingBinding(binding.ResourcePath, storedSpec) + existingBinding, err := findExistingBinding(binding.ResourcePath, req.LastMaterialization) if err != nil { return nil, fmt.Errorf("finding existing binding: %w", err) } @@ -147,7 +121,7 @@ func ApplyChanges(ctx context.Context, req *pm.Request_Apply, applier Applier, i desc = append(desc, createDesc) } - action = func(ctx context.Context) error { + action := func(ctx context.Context) error { if err := deleteAction(ctx); err != nil { return err } @@ -241,19 +215,5 @@ func ApplyChanges(ctx context.Context, req *pm.Request_Apply, applier Applier, i } } - // Only update the spec after all other actions have completed successfully. - desc, action, err = applier.PutSpec(ctx, req.Materialization, req.Version, storedSpec != nil) - if err != nil { - return nil, fmt.Errorf("getting PutSpec action: %w", err) - } - // Although all current materializations always do persist a spec, its possible that some may - // not in the future as we transition to runtime provided specs for apply. - if action != nil { - actionDescriptions = append(actionDescriptions, desc) - if err := action(ctx); err != nil { - return nil, fmt.Errorf("updating persisted specification: %w", err) - } - } - return &pm.Response_Applied{ActionDescription: strings.Join(actionDescriptions, "\n")}, nil } diff --git a/materialize-boilerplate/apply_test.go b/materialize-boilerplate/apply_test.go index d4bac11a85..922597bc39 100644 --- a/materialize-boilerplate/apply_test.go +++ b/materialize-boilerplate/apply_test.go @@ -52,62 +52,37 @@ func TestApply(t *testing.T) { name: "new materialization", originalSpec: nil, newSpec: loadApplySpec(t, "base.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - createdResources: 1, - }, + want: testResults{createdResources: 1}, }, { name: "remove required field", originalSpec: loadApplySpec(t, "base.flow.proto"), newSpec: loadApplySpec(t, "remove-required.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - nullabledProjections: 1, - }, + want: testResults{nullabledProjections: 1}, }, { name: "add required field", originalSpec: loadApplySpec(t, "base.flow.proto"), newSpec: loadApplySpec(t, "add-new-required.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - addedProjections: 1, - }, + want: testResults{addedProjections: 1}, }, { name: "add binding", originalSpec: loadApplySpec(t, "base.flow.proto"), newSpec: loadApplySpec(t, "add-new-binding.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - createdResources: 1, - }, + want: testResults{createdResources: 1}, }, { name: "replace binding", originalSpec: loadApplySpec(t, "base.flow.proto"), newSpec: loadApplySpec(t, "replace-original-binding.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - deletedResources: 1, - createdResources: 1, - }, + want: testResults{deletedResources: 1, createdResources: 1}, }, { name: "field is newly nullable", originalSpec: loadApplySpec(t, "base.flow.proto"), newSpec: loadApplySpec(t, "make-nullable.flow.proto"), - want: testResults{ - createdMetaTables: true, - putSpec: true, - nullabledProjections: 1, - }, + want: testResults{nullabledProjections: 1}, }, } @@ -120,7 +95,7 @@ func TestApply(t *testing.T) { } is := testInfoSchemaFromSpec(t, tt.originalSpec, simpleTestTransform) - req := &pm.Request_Apply{Materialization: tt.newSpec, Version: "aVersion"} + req := &pm.Request_Apply{Materialization: tt.newSpec, Version: "aVersion", LastMaterialization: tt.originalSpec} // Not concurrent. got, err := ApplyChanges(ctx, req, app, is, false) @@ -147,8 +122,6 @@ func TestApply(t *testing.T) { } type testResults struct { - createdMetaTables bool - putSpec bool createdResources int deletedResources int addedProjections int @@ -164,17 +137,10 @@ type testApplier struct { results testResults } -func (a *testApplier) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, ActionApplyFn, error) { - return "create meta tables", func(ctx context.Context) error { - a.results.createdMetaTables = true - return nil - }, nil -} - func (a *testApplier) CreateResource(ctx context.Context, spec *pf.MaterializationSpec, bindingIndex int) (string, ActionApplyFn, error) { binding := spec.Bindings[bindingIndex] - return fmt.Sprintf("create resource for collection %q", binding.Collection.Name.String()), func(ctx context.Context) error { + return fmt.Sprintf("create resource for %q", binding.ResourcePath), func(ctx context.Context) error { a.mu.Lock() defer a.mu.Unlock() @@ -183,17 +149,6 @@ func (a *testApplier) CreateResource(ctx context.Context, spec *pf.Materializati }, nil } -func (a *testApplier) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - return a.storedSpec, nil -} - -func (a *testApplier) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, exists bool) (string, ActionApplyFn, error) { - return fmt.Sprintf("put spec with version %q", version), func(ctx context.Context) error { - a.results.putSpec = true - return nil - }, nil -} - func (a *testApplier) DeleteResource(ctx context.Context, path []string) (string, ActionApplyFn, error) { return fmt.Sprintf("delete resource %q", path), func(ctx context.Context) error { a.mu.Lock() @@ -214,8 +169,8 @@ func (a *testApplier) UpdateResource(ctx context.Context, spec *pf.Materializati } action := fmt.Sprintf( - "update resource for collection %q [new projections: %d, newly nullable fields: %d, newly delta updates: %t]", - binding.Collection.Name.String(), + "update resource for %q [new projections: %d, newly nullable fields: %d, newly delta updates: %t]", + binding.ResourcePath, len(bindingUpdate.NewProjections), len(bindingUpdate.NewlyNullableFields), bindingUpdate.NewlyDeltaUpdates, diff --git a/materialize-boilerplate/test_support.go b/materialize-boilerplate/test_support.go index afdcfb3f2e..fb0371365e 100644 --- a/materialize-boilerplate/test_support.go +++ b/materialize-boilerplate/test_support.go @@ -45,7 +45,7 @@ func RunValidateAndApplyTestCases( config any, resourceConfig any, dumpSchema func(t *testing.T) string, - cleanup func(t *testing.T, materialization pf.Materialization), + cleanup func(t *testing.T), ) { ctx := context.Background() var snap strings.Builder @@ -57,7 +57,7 @@ func RunValidateAndApplyTestCases( require.NoError(t, err) t.Run("validate and apply many different types of fields", func(t *testing.T) { - defer cleanup(t, pf.Materialization("test/sqlite")) + defer cleanup(t) fixture := loadSpec(t, "big-schema.flow.proto") @@ -155,7 +155,7 @@ func RunValidateAndApplyTestCases( for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - defer cleanup(t, pf.Materialization("test/sqlite")) + defer cleanup(t) initial := loadSpec(t, "base.flow.proto") @@ -178,7 +178,7 @@ func RunValidateAndApplyTestCases( }) t.Run("validate and apply fields with challenging names", func(t *testing.T) { - defer cleanup(t, pf.Materialization("test/sqlite")) + defer cleanup(t) fixture := loadSpec(t, "challenging-fields.flow.proto") diff --git a/materialize-boilerplate/testdata/validate_apply_test_cases/add-and-remove-many.flow.yaml b/materialize-boilerplate/testdata/validate_apply_test_cases/add-and-remove-many.flow.yaml index 8cbe7d194a..8e600c03c5 100644 --- a/materialize-boilerplate/testdata/validate_apply_test_cases/add-and-remove-many.flow.yaml +++ b/materialize-boilerplate/testdata/validate_apply_test_cases/add-and-remove-many.flow.yaml @@ -14,7 +14,7 @@ collections: optionalObject: { type: object } addedOptionalString: { type: string } addedRequiredString: { type: string } - required: [key, requiredString, requiredInteger, requiredBoolean, requiredObject, addedRequiredString] + required: [key, requiredInteger, requiredBoolean, addedRequiredString] key: [/key] projections: flow_document: "" diff --git a/materialize-boilerplate/testdata/validate_apply_test_cases/generated_specs/add-and-remove-many.flow.proto b/materialize-boilerplate/testdata/validate_apply_test_cases/generated_specs/add-and-remove-many.flow.proto index e4512ba95e..496037bb78 100644 Binary files a/materialize-boilerplate/testdata/validate_apply_test_cases/generated_specs/add-and-remove-many.flow.proto and b/materialize-boilerplate/testdata/validate_apply_test_cases/generated_specs/add-and-remove-many.flow.proto differ diff --git a/materialize-boilerplate/validate.go b/materialize-boilerplate/validate.go index d1abc3e2af..0af39a8011 100644 --- a/materialize-boilerplate/validate.go +++ b/materialize-boilerplate/validate.go @@ -63,9 +63,9 @@ func (v Validator) ValidateBinding( backfill uint32, boundCollection pf.CollectionSpec, fieldConfigJsonMap map[string]json.RawMessage, - storedSpec *pf.MaterializationSpec, + lastSpec *pf.MaterializationSpec, ) (map[string]*pm.Response_Validated_Constraint, error) { - existingBinding, err := findExistingBinding(path, storedSpec) + existingBinding, err := findExistingBinding(path, lastSpec) if err != nil { return nil, err } @@ -389,12 +389,12 @@ func (v Validator) ambiguousFieldIsSelected(p pf.Projection, fieldSelection []st return false } -// findExistingBinding locates a binding within an existing stored specification. -func findExistingBinding(resourcePath []string, storedSpec *pf.MaterializationSpec) (*pf.MaterializationSpec_Binding, error) { - if storedSpec == nil { +// findExistingBinding locates a binding within an previously applied or validated specification. +func findExistingBinding(resourcePath []string, lastSpec *pf.MaterializationSpec) (*pf.MaterializationSpec_Binding, error) { + if lastSpec == nil { return nil, nil // Binding is trivially not found } - for _, existingBinding := range storedSpec.Bindings { + for _, existingBinding := range lastSpec.Bindings { if slices.Equal(resourcePath, existingBinding.ResourcePath) { return existingBinding, nil } diff --git a/materialize-databricks/.snapshots/TestValidateAndApply b/materialize-databricks/.snapshots/TestValidateAndApply index b990d4fb45..debc19e17d 100644 --- a/materialize-databricks/.snapshots/TestValidateAndApply +++ b/materialize-databricks/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BOOLEAN' but endpoint type 'LONG' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'LONG' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'STRING' but endpoint type 'LONG' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'STRING' is required by its schema '{ type: [string] }'"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-databricks/.snapshots/TestValidateAndApplyMigrations b/materialize-databricks/.snapshots/TestValidateAndApplyMigrations index 97cadb4578..27405c2311 100644 --- a/materialize-databricks/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-databricks/.snapshots/TestValidateAndApplyMigrations @@ -41,8 +41,7 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (STRING), _meta/flow_truncated (BOOLEAN), boolWidenedToJson (BOOLEAN), dateValue (DATE), datetimeValue (TIMESTAMP), flow_published_at (TIMESTAMP), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (STRING), nonScalarValue (STRING), numericString (DECIMAL), optional (STRING), requiredNumeric (DECIMAL), scalarValue (STRING), stringWidenedToJson (STRING), timeValue (STRING), flow_document (STRING) - -1, false, true, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.111111 +0000 UTC, 2024-09-13 01:01:01 +0000 UTC, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} +1, false, true, 2024-01-01T00:00:00Z, 2024-01-01T01:01:01.111111Z, 2024-09-13T01:01:01Z, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (STRING), _meta/flow_truncated (BOOLEAN), boolWidenedToJson (BOOLEAN), flow_published_at (TIMESTAMP), int64 (BIGINT), multiple (STRING), nonScalarValue (STRING), optional (STRING), scalarValue (STRING), stringWidenedToJson (STRING), timeValue (STRING), flow_document (STRING), dateValue (STRING), datetimeValue (STRING), intWidenedToJson (STRING), numericString (STRING), requiredNumeric (STRING) - -1, false, true, 2024-09-13 01:01:01 +0000 UTC, 1, , , , test, hello, 01:01:01, {}, 2024-01-01, 2024-01-01T01:01:01.111111000Z, 999, 123, 456 +1, false, true, 2024-09-13T01:01:01Z, 1, , , , test, hello, 01:01:01, {}, 2024-01-01, 2024-01-01T01:01:01.111111000Z, 999, 123, 456 diff --git a/materialize-databricks/client.go b/materialize-databricks/client.go index 0b13227d2a..186b5957a3 100644 --- a/materialize-databricks/client.go +++ b/materialize-databricks/client.go @@ -18,7 +18,6 @@ import ( dbsqlerr "github.com/databricks/databricks-sql-go/errors" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" log "github.com/sirupsen/logrus" _ "github.com/databricks/databricks-sql-go" @@ -123,11 +122,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return is, nil } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.QueryString) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { _, err := c.db.ExecContext(ctx, tc.TableCreateSql) return err @@ -303,23 +297,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (string, string, error) { - var version, spec string - - if err := c.db.QueryRowContext( - ctx, - fmt.Sprintf( - "SELECT version, spec FROM %s WHERE materialization = %s;", - specs.Identifier, - databricksDialect.Literal(materialization.String()), - ), - ).Scan(&version, &spec); err != nil { - return "", "", err - } - - return spec, version, nil -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-databricks/driver.go b/materialize-databricks/driver.go index 7bbe42840d..7469ab89e7 100644 --- a/materialize-databricks/driver.go +++ b/materialize-databricks/driver.go @@ -86,13 +86,9 @@ func newDatabricksDriver() *sql.Driver { "catalog": cfg.CatalogName, }).Info("connecting to databricks") - var metaBase sql.TablePath = []string{cfg.SchemaName} - var metaSpecs, _ = sql.MetaTables(metaBase) - return &sql.Endpoint{ Config: cfg, Dialect: databricksDialect, - MetaSpecs: &metaSpecs, MetaCheckpoints: nil, NewClient: newClient, CreateTableTemplate: tplCreateTargetTable, diff --git a/materialize-databricks/driver_test.go b/materialize-databricks/driver_test.go index 61504d7c11..a895f1452f 100644 --- a/materialize-databricks/driver_test.go +++ b/materialize-databricks/driver_test.go @@ -12,7 +12,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" "github.com/stretchr/testify/require" _ "github.com/databricks/databricks-sql-go" @@ -75,16 +74,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", databricksDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - databricksDialect.Identifier(cfg.SchemaName, sql.DefaultFlowMaterializations), - databricksDialect.Literal(materialization.String()), - )) }, ) } @@ -143,16 +135,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", databricksDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - databricksDialect.Identifier(sql.DefaultFlowMaterializations), - databricksDialect.Literal(materialization.String()), - )) }, ) } diff --git a/materialize-dynamodb/apply.go b/materialize-dynamodb/apply.go index d2400b63e9..0e843f3e86 100644 --- a/materialize-dynamodb/apply.go +++ b/materialize-dynamodb/apply.go @@ -17,14 +17,6 @@ import ( type ddbApplier struct { client *client cfg config - // TODO(whb): Including the lastSpec from the validate or apply request is a - // temporary hack until we get around to removing the "load/persist a spec - // in the destination" concept more thoroughly. - lastSpec *pf.MaterializationSpec -} - -func (e *ddbApplier) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, boilerplate.ActionApplyFn, error) { - return "", nil, nil } func (e *ddbApplier) CreateResource(ctx context.Context, spec *pf.MaterializationSpec, bindingIndex int) (string, boilerplate.ActionApplyFn, error) { @@ -38,14 +30,6 @@ func (e *ddbApplier) CreateResource(ctx context.Context, spec *pf.Materializatio }, nil } -func (e *ddbApplier) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - return e.lastSpec, nil -} - -func (e *ddbApplier) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, _ bool) (string, boilerplate.ActionApplyFn, error) { - return "", nil, nil -} - func (e *ddbApplier) DeleteResource(ctx context.Context, path []string) (string, boilerplate.ActionApplyFn, error) { return fmt.Sprintf("delete table %q", path[0]), func(ctx context.Context) error { return deleteTable(ctx, e.client, path[0]) diff --git a/materialize-dynamodb/driver.go b/materialize-dynamodb/driver.go index eb3ee01436..93fffda6b9 100644 --- a/materialize-dynamodb/driver.go +++ b/materialize-dynamodb/driver.go @@ -262,9 +262,8 @@ func (d driver) Apply(ctx context.Context, req *pm.Request_Apply) (*pm.Response_ } return boilerplate.ApplyChanges(ctx, req, &ddbApplier{ - client: client, - cfg: cfg, - lastSpec: req.LastMaterialization, + client: client, + cfg: cfg, }, is, true) } diff --git a/materialize-dynamodb/driver_test.go b/materialize-dynamodb/driver_test.go index 56c9123d0f..a0a0af788e 100644 --- a/materialize-dynamodb/driver_test.go +++ b/materialize-dynamodb/driver_test.go @@ -16,7 +16,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/stretchr/testify/require" ) @@ -71,7 +70,7 @@ func TestValidateAndApply(t *testing.T) { return out.String() }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() for _, table := range []string{resourceConfig.Table} { diff --git a/materialize-elasticsearch/apply.go b/materialize-elasticsearch/apply.go index 677594acfe..b3b16f60d3 100644 --- a/materialize-elasticsearch/apply.go +++ b/materialize-elasticsearch/apply.go @@ -14,12 +14,6 @@ type elasticApplier struct { cfg config } -func (e *elasticApplier) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, boilerplate.ActionApplyFn, error) { - return fmt.Sprintf("create index %q", defaultFlowMaterializations), func(ctx context.Context) error { - return e.client.createMetaIndex(ctx, e.cfg.Advanced.Replicas) - }, nil -} - func (e *elasticApplier) CreateResource(ctx context.Context, spec *pf.MaterializationSpec, bindingIndex int) (string, boilerplate.ActionApplyFn, error) { binding := spec.Bindings[bindingIndex] @@ -38,16 +32,6 @@ func (e *elasticApplier) CreateResource(ctx context.Context, spec *pf.Materializ }, nil } -func (e *elasticApplier) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - return e.client.getSpec(ctx, materialization) -} - -func (e *elasticApplier) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, _ bool) (string, boilerplate.ActionApplyFn, error) { - return fmt.Sprintf("update stored materialization spec and set version = %s", version), func(ctx context.Context) error { - return e.client.putSpec(ctx, spec, version) - }, nil -} - func (e *elasticApplier) DeleteResource(ctx context.Context, path []string) (string, boilerplate.ActionApplyFn, error) { return fmt.Sprintf("delete index %q", path[0]), func(ctx context.Context) error { return e.client.deleteIndex(ctx, path[0]) diff --git a/materialize-elasticsearch/client.go b/materialize-elasticsearch/client.go index ee22069f9c..481d49c17c 100644 --- a/materialize-elasticsearch/client.go +++ b/materialize-elasticsearch/client.go @@ -2,102 +2,20 @@ package main import ( "context" - "encoding/base64" "encoding/json" "fmt" - "io" "net/http" - "net/url" "strings" elasticsearch "github.com/elastic/go-elasticsearch/v8" "github.com/elastic/go-elasticsearch/v8/esutil" boilerplate "github.com/estuary/connectors/materialize-boilerplate" - pf "github.com/estuary/flow/go/protocols/flow" - "github.com/tidwall/gjson" -) - -const ( - defaultFlowMaterializations = "flow_materializations_v2" ) type client struct { es *elasticsearch.Client } -func (c *client) createMetaIndex(ctx context.Context, replicas *int) error { - props := map[string]property{ - "version": {Type: elasticTypeKeyword, Index: boolPtr(false)}, - // Binary mappings are never indexed, and to specify index: false on such a mapping results - // in an error. - "specBytes": {Type: elasticTypeBinary}, - } - - // The meta index will always be created with the default number of shards. - // Long term we plan to not require a meta index at all. - return c.createIndex(ctx, defaultFlowMaterializations, nil, replicas, props) -} - -func (c *client) putSpec(ctx context.Context, spec *pf.MaterializationSpec, version string) error { - specBytes, err := spec.Marshal() - if err != nil { - return fmt.Errorf("marshalling spec: %w", err) - } - - res, err := c.es.Index( - defaultFlowMaterializations, - esutil.NewJSONReader(map[string]string{ - "specBytes": base64.StdEncoding.EncodeToString(specBytes), - "version": version, - }), - c.es.Index.WithContext(ctx), - c.es.Index.WithDocumentID(url.PathEscape(spec.Name.String())), - ) - if err != nil { - return fmt.Errorf("putSpec: %w", err) - } - defer res.Body.Close() - if res.IsError() { - return fmt.Errorf("putSpec error response [%s] %s", res.Status(), res.String()) - } - - return nil -} - -func (c *client) getSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - res, err := c.es.Get( - defaultFlowMaterializations, - url.PathEscape(string(materialization)), - c.es.Get.WithContext(ctx), - ) - if err != nil { - return nil, fmt.Errorf("getSpec spec: %w", err) - } - defer res.Body.Close() - - if res.StatusCode == http.StatusNotFound { - return nil, nil - } else if res.IsError() { - return nil, fmt.Errorf("getSpec error response [%s] %s", res.Status(), res.String()) - } - - var spec pf.MaterializationSpec - - if jsonBytes, err := io.ReadAll(res.Body); err != nil { - return nil, fmt.Errorf("reading response body: %w", err) - } else if loc := gjson.GetBytes(jsonBytes, "_source.specBytes"); !loc.Exists() { - return nil, fmt.Errorf("malformed response: '_source.specBytes' does not exist") - } else if specBytes, err := base64.StdEncoding.DecodeString(loc.String()); err != nil { - return nil, fmt.Errorf("base64.Decode: %w", err) - } else if err := spec.Unmarshal(specBytes); err != nil { - return nil, fmt.Errorf("spec.Unmarshal: %w", err) - } else if err := spec.Validate(); err != nil { - return nil, fmt.Errorf("validating spec: %w", err) - } - - return &spec, nil -} - type createIndexParams struct { Settings indexSettings `json:"settings,omitempty"` Mappings indexMappings `json:"mappings"` diff --git a/materialize-elasticsearch/driver.go b/materialize-elasticsearch/driver.go index fede893bc5..10a83b4bcb 100644 --- a/materialize-elasticsearch/driver.go +++ b/materialize-elasticsearch/driver.go @@ -392,11 +392,6 @@ func (driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Respo return nil, cerrors.NewUserError(nil, fmt.Sprintf("could not connect to endpoint: received status code %d", p.StatusCode)) } - storedSpec, err := client.getSpec(ctx, req.Name) - if err != nil { - return nil, fmt.Errorf("getting spec: %w", err) - } - is, err := client.infoSchema(ctx) if err != nil { return nil, fmt.Errorf("getting infoSchema for validate: %w", err) @@ -422,7 +417,7 @@ func (driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Respo binding.Backfill, binding.Collection, binding.FieldConfigJsonMap, - storedSpec, + req.LastMaterialization, ) if err != nil { return nil, fmt.Errorf("validating binding: %w", err) diff --git a/materialize-elasticsearch/driver_test.go b/materialize-elasticsearch/driver_test.go index 51f815a4d7..d45c9f521b 100644 --- a/materialize-elasticsearch/driver_test.go +++ b/materialize-elasticsearch/driver_test.go @@ -13,7 +13,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" @@ -82,10 +81,9 @@ func TestValidateAndApply(t *testing.T) { return out.String() }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - - _, err := client.es.Indices.Delete([]string{defaultFlowMaterializations, resourceConfig.Index}) + _, err := client.es.Indices.Delete([]string{resourceConfig.Index}) require.NoError(t, err) }, ) diff --git a/materialize-mongodb/apply.go b/materialize-mongodb/apply.go index 421cd777b4..d6852b5a73 100644 --- a/materialize-mongodb/apply.go +++ b/materialize-mongodb/apply.go @@ -6,10 +6,7 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" pf "github.com/estuary/flow/go/protocols/flow" - proto "github.com/gogo/protobuf/proto" - "go.mongodb.org/mongo-driver/bson" "go.mongodb.org/mongo-driver/mongo" - "go.mongodb.org/mongo-driver/mongo/options" ) const specCollection = "flow_materializations" @@ -19,56 +16,11 @@ type mongoApplier struct { cfg config } -func (e *mongoApplier) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, boilerplate.ActionApplyFn, error) { - // No-op since new collections are automatically created when data is added to them. - return "", nil, nil -} - func (e *mongoApplier) CreateResource(ctx context.Context, spec *pf.MaterializationSpec, bindingIndex int) (string, boilerplate.ActionApplyFn, error) { // No-op since new collections are automatically created when data is added to them. return "", nil, nil } -func (a *mongoApplier) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - metaCollection := a.client.Database(a.cfg.Database).Collection(specCollection) - - type savedSpec struct { - Id string `bson:"uuid"` - Spec []byte `bson:"spec"` - } - - var s savedSpec - if err := metaCollection.FindOne(ctx, bson.D{{Key: idField, Value: bson.D{{Key: "$eq", Value: materialization}}}}).Decode(&s); err != nil { - if err == mongo.ErrNoDocuments { - return nil, nil - } - return nil, fmt.Errorf("could not find spec: %w", err) - } - - var existing pf.MaterializationSpec - if err := proto.Unmarshal(s.Spec, &existing); err != nil { - return nil, fmt.Errorf("parsing existing materialization spec: %w", err) - } - - return &existing, nil -} - -func (a *mongoApplier) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, _ bool) (string, boilerplate.ActionApplyFn, error) { - bs, err := proto.Marshal(spec) - if err != nil { - return "", nil, fmt.Errorf("encoding existing materialization spec: %w", err) - } - - return "update persisted spec", func(ctx context.Context) error { - metaCollection := a.client.Database(a.cfg.Database).Collection(specCollection) - opts := options.Replace().SetUpsert(true) - if _, err := metaCollection.ReplaceOne(ctx, bson.D{{Key: idField, Value: spec.Name.String()}}, bson.D{{Key: "spec", Value: bs}}, opts); err != nil { - return fmt.Errorf("upserting spec: %w", err) - } - return nil - }, nil -} - func (a *mongoApplier) DeleteResource(ctx context.Context, path []string) (string, boilerplate.ActionApplyFn, error) { return fmt.Sprintf("drop collection %q", path[1]), func(ctx context.Context) error { return a.client.Database(path[0]).Collection(path[1]).Drop(ctx) diff --git a/materialize-motherduck/.snapshots/TestValidateAndApply b/materialize-motherduck/.snapshots/TestValidateAndApply index f23d70b048..1f85bccb82 100644 --- a/materialize-motherduck/.snapshots/TestValidateAndApply +++ b/materialize-motherduck/.snapshots/TestValidateAndApply @@ -82,13 +82,13 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BOOLEAN' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'BIGINT' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'objField' is already being materialized as endpoint type 'JSON' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP WITH TIME ZONE' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} {"Field":"stringDurationField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDurationField' is already being materialized as endpoint type 'INTERVAL' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -96,7 +96,7 @@ Big Schema Changed Types Constraints: {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'HUGEINT' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringTimeField' is already being materialized as endpoint type 'TIME' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'HUGEINT' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'HUGEINT' but endpoint type 'VARCHAR' is required by its schema '{ type: [string] }'"} +{"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-motherduck/.snapshots/TestValidateAndApplyMigrations b/materialize-motherduck/.snapshots/TestValidateAndApplyMigrations index fb62e5a2e2..db79a0bbba 100644 --- a/materialize-motherduck/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-motherduck/.snapshots/TestValidateAndApplyMigrations @@ -42,8 +42,7 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (VARCHAR), _meta/flow_truncated (BOOLEAN), boolWidenedToJson (BOOLEAN), dateValue (DATE), datetimeValue (TIMESTAMPTZ), flow_document (VARCHAR), flow_published_at (TIMESTAMPTZ), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (VARCHAR), nonScalarValue (VARCHAR), numericString (HUGEINT), optional (VARCHAR), requiredNumeric (HUGEINT), scalarValue (VARCHAR), stringWidenedToJson (VARCHAR), timeValue (TIME), second_root (VARCHAR) - -1, false, true, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.111111 +0000 UTC, {}, 2024-09-13 01:01:01 +0000 UTC, 1, 999, , , 123, , 456, test, hello, 1970-01-01 01:01:01 +0000 UTC, {} +1, false, true, 2024-01-01T00:00:00Z, 2024-01-01T01:01:01.111111Z, {}, 2024-09-13T01:01:01Z, 1, 999, , , 123, , 456, test, hello, 1970-01-01T01:01:01Z, {} Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -89,6 +88,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (VARCHAR), _meta/flow_truncated (BOOLEAN), datetimeValue (TIMESTAMPTZ), flow_document (VARCHAR), flow_published_at (TIMESTAMPTZ), int64 (BIGINT), multiple (VARCHAR), nonScalarValue (VARCHAR), optional (VARCHAR), scalarValue (VARCHAR), second_root (VARCHAR), boolWidenedToJson (VARCHAR), dateValue (VARCHAR), intWidenedToJson (VARCHAR), numericString (VARCHAR), requiredNumeric (VARCHAR), stringWidenedToJson (VARCHAR), timeValue (VARCHAR) - -1, false, 2024-01-01 01:01:01.111111 +0000 UTC, {}, 2024-09-13 01:01:01 +0000 UTC, 1, , , , test, {}, true, 2024-01-01, 999, 123, 456, "hello", 01:01:01 +1, false, 2024-01-01T01:01:01.111111Z, {}, 2024-09-13T01:01:01Z, 1, , , , test, {}, true, 2024-01-01, 999, 123, 456, "hello", 01:01:01 diff --git a/materialize-motherduck/client.go b/materialize-motherduck/client.go index cc59f7bcab..4bd11fcd3c 100644 --- a/materialize-motherduck/client.go +++ b/materialize-motherduck/client.go @@ -15,7 +15,6 @@ import ( awsHttp "github.com/aws/smithy-go/transport/http" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" "github.com/google/uuid" _ "github.com/marcboeker/go-duckdb" @@ -48,11 +47,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return sql.StdFetchInfoSchema(ctx, c.db, c.ep.Dialect, c.cfg.Database, resourcePaths) } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.QueryString) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { _, err := c.db.ExecContext(ctx, tc.TableCreateSql) return err @@ -202,10 +196,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (string, string, error) { - return sql.StdFetchSpecAndVersion(ctx, c.db, specs, materialization) -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-motherduck/driver.go b/materialize-motherduck/driver.go index 031522ad5c..b7fab8b155 100644 --- a/materialize-motherduck/driver.go +++ b/materialize-motherduck/driver.go @@ -163,13 +163,10 @@ func newDuckDriver() *sql.Driver { "database": cfg.Database, }).Info("opening database") - metaSpecs, metaCheckpoints := sql.MetaTables([]string{cfg.Database, cfg.Schema}) - return &sql.Endpoint{ Config: cfg, Dialect: duckDialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable([]string{cfg.Database, cfg.Schema}), NewClient: newClient, CreateTableTemplate: tplCreateTargetTable, NewResource: newTableConfig, diff --git a/materialize-motherduck/driver_test.go b/materialize-motherduck/driver_test.go index 604257fc89..0fbd1dcd18 100644 --- a/materialize-motherduck/driver_test.go +++ b/materialize-motherduck/driver_test.go @@ -11,7 +11,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/google/uuid" "github.com/stretchr/testify/require" @@ -76,7 +75,7 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() db, err := cfg.db(ctx) @@ -84,12 +83,6 @@ func TestValidateAndApply(t *testing.T) { defer db.Close() _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", duckDialect.Identifier(cfg.Database, resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - duckDialect.Identifier(cfg.Database, cfg.Schema, sql.DefaultFlowMaterializations), - duckDialect.Literal(materialization.String()), - )) }, ) } @@ -157,7 +150,7 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() db, err := cfg.db(ctx) @@ -165,12 +158,6 @@ func TestValidateAndApplyMigrations(t *testing.T) { defer db.Close() _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", duckDialect.Identifier(cfg.Database, resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - duckDialect.Identifier(cfg.Database, cfg.Schema, sql.DefaultFlowMaterializations), - duckDialect.Literal(materialization.String()), - )) }, ) } diff --git a/materialize-mysql/.snapshots/TestValidateAndApply b/materialize-mysql/.snapshots/TestValidateAndApply index 5a8be21202..b1413fb791 100644 --- a/materialize-mysql/.snapshots/TestValidateAndApply +++ b/materialize-mysql/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'TINYINT' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'BIGINT' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'objField' is already being materialized as endpoint type 'JSON' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'DATETIME' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'LONGTEXT' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'DOUBLE' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringTimeField' is already being materialized as endpoint type 'TIME' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'DECIMAL' but endpoint type 'LONGTEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-mysql/.snapshots/TestValidateAndApplyMigrations b/materialize-mysql/.snapshots/TestValidateAndApplyMigrations index afb0ccd91c..4e23f60672 100644 --- a/materialize-mysql/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-mysql/.snapshots/TestValidateAndApplyMigrations @@ -41,7 +41,6 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (VARCHAR), _meta/flow_truncated (TINYINT), boolWidenedToJson (TINYINT), dateValue (DATE), datetimeValue (DATETIME), flow_published_at (DATETIME), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (JSON), nonScalarValue (JSON), numericString (DECIMAL), optional (JSON), requiredNumeric (DECIMAL), scalarValue (TEXT), stringWidenedToJson (TEXT), timeValue (TIME), flow_document (JSON) - 1, 0, 1, 2024-01-01, 2024-01-01 01:01:01.111111, 2024-09-13 01:01:01.000000, 1, 999, , , 123, , 456, test, hello, 01:01:01.000000, {} Migratable Changes Constraints: @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (VARCHAR), _meta/flow_truncated (TINYINT), flow_published_at (DATETIME), int64 (BIGINT), multiple (JSON), nonScalarValue (JSON), optional (JSON), scalarValue (TEXT), flow_document (JSON), boolWidenedToJson (JSON), dateValue (TEXT), datetimeValue (TEXT), intWidenedToJson (JSON), numericString (TEXT), requiredNumeric (TEXT), stringWidenedToJson (JSON), timeValue (TEXT) - 1, 0, 2024-09-13 01:01:01.000000, 1, , , , test, {}, 1, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, "hello", 01:01:01.000000 diff --git a/materialize-mysql/.snapshots/TestValidateAndApplyMigrationsMariaDB b/materialize-mysql/.snapshots/TestValidateAndApplyMigrationsMariaDB index df9c9a47e0..18451458f7 100644 --- a/materialize-mysql/.snapshots/TestValidateAndApplyMigrationsMariaDB +++ b/materialize-mysql/.snapshots/TestValidateAndApplyMigrationsMariaDB @@ -41,7 +41,6 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (VARCHAR), _meta/flow_truncated (TINYINT), boolWidenedToJson (TINYINT), dateValue (DATE), datetimeValue (DATETIME), flow_published_at (DATETIME), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (TEXT), nonScalarValue (TEXT), numericString (DECIMAL), optional (TEXT), requiredNumeric (DECIMAL), scalarValue (TEXT), stringWidenedToJson (TEXT), timeValue (TIME), flow_document (TEXT) - 1, 0, 1, 2024-01-01, 2024-01-01 01:01:01.111111, 2024-09-13 01:01:01.000000, 1, 999, , , 123, , 456, test, hello, 01:01:01.000000, {} Migratable Changes Constraints: @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (VARCHAR), _meta/flow_truncated (TINYINT), boolWidenedToJson (TINYINT), flow_published_at (DATETIME), int64 (BIGINT), multiple (TEXT), nonScalarValue (TEXT), optional (TEXT), scalarValue (TEXT), stringWidenedToJson (TEXT), flow_document (TEXT), dateValue (TEXT), datetimeValue (TEXT), intWidenedToJson (TEXT), numericString (TEXT), requiredNumeric (TEXT), timeValue (TEXT) - 1, 0, 1, 2024-09-13 01:01:01.000000, 1, , , , test, hello, {}, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, 01:01:01.000000 diff --git a/materialize-mysql/client.go b/materialize-mysql/client.go index b381812b33..6b27e62d34 100644 --- a/materialize-mysql/client.go +++ b/materialize-mysql/client.go @@ -11,7 +11,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - "github.com/estuary/flow/go/protocols/flow" "github.com/go-sql-driver/mysql" ) @@ -238,11 +237,6 @@ func (c *client) DeleteTable(ctx context.Context, path []string) (string, boiler }, nil } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - func (c *client) InstallFence(ctx context.Context, checkpoints sql.Table, fence sql.Fence) (sql.Fence, error) { return sql.StdInstallFence(ctx, c.db, checkpoints, fence) } @@ -251,10 +245,6 @@ func (c *client) ExecStatements(ctx context.Context, statements []string) error return sql.StdSQLExecStatements(ctx, c.db, statements) } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization flow.Materialization) (string, string, error) { - return sql.StdFetchSpecAndVersion(ctx, c.db, specs, materialization) -} - func (c *client) Close() { c.db.Close() } diff --git a/materialize-mysql/driver.go b/materialize-mysql/driver.go index 21d026b883..a967579dbf 100644 --- a/materialize-mysql/driver.go +++ b/materialize-mysql/driver.go @@ -263,9 +263,6 @@ func newMysqlDriver() *sql.Driver { "user": cfg.User, }).Info("opening database") - var metaBase sql.TablePath - var metaSpecs, metaCheckpoints = sql.MetaTables(metaBase) - if cfg.Advanced.SSLMode == "verify_ca" || cfg.Advanced.SSLMode == "verify_identity" { if err := registerCustomSSL(cfg); err != nil { return nil, fmt.Errorf("error registering custom ssl: %w", err) @@ -325,8 +322,7 @@ func newMysqlDriver() *sql.Driver { return &sql.Endpoint{ Config: cfg, Dialect: dialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable(nil), NewClient: prepareNewClient(tzLocation), CreateTableTemplate: templates.createTargetTable, NewResource: newTableConfig, diff --git a/materialize-mysql/driver_test.go b/materialize-mysql/driver_test.go index 86b343b25a..7782614607 100644 --- a/materialize-mysql/driver_test.go +++ b/materialize-mysql/driver_test.go @@ -71,15 +71,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = 'test/sqlite'", - testDialect.Identifier(sql.DefaultFlowMaterializations), - )) }, ) } @@ -137,16 +131,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -204,16 +191,9 @@ func TestValidateAndApplyMigrationsMariaDB(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -364,10 +344,10 @@ func TestPrereqs(t *testing.T) { { name: "wrong address", cfg: func(cfg config) *config { - cfg.Address = "wrong." + cfg.Address + cfg.Address = cfg.Address + ".wrong" return &cfg }, - want: []string{fmt.Sprintf("host at address %q cannot be found", "wrong."+cfg.Address)}, + want: []string{fmt.Sprintf("host at address %q cannot be found", cfg.Address+".wrong")}, }, } diff --git a/materialize-postgres/.snapshots/TestValidateAndApply b/materialize-postgres/.snapshots/TestValidateAndApply index 1eda591c41..c75116bd26 100644 --- a/materialize-postgres/.snapshots/TestValidateAndApply +++ b/materialize-postgres/.snapshots/TestValidateAndApply @@ -88,15 +88,15 @@ Big Schema Changed Types Constraints: {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'DOUBLE PRECISION' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'objField' is already being materialized as endpoint type 'JSON' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP WITH TIME ZONE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDurationField' is already being materialized as endpoint type 'INTERVAL' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'TEXT' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIpv4Field' is already being materialized as endpoint type 'CIDR' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} {"Field":"stringIpv6Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIpv6Field' is already being materialized as endpoint type 'CIDR' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringMacAddr8Field' is already being materialized as endpoint type 'MACADDR8' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} {"Field":"stringMacAddrField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringMacAddrField' is already being materialized as endpoint type 'MACADDR' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringTimeField' is already being materialized as endpoint type 'TIME WITHOUT TIME ZONE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-postgres/.snapshots/TestValidateAndApplyMigrations b/materialize-postgres/.snapshots/TestValidateAndApplyMigrations index a7d5853c3b..d3f68e6be5 100644 --- a/materialize-postgres/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-postgres/.snapshots/TestValidateAndApplyMigrations @@ -41,8 +41,7 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (TEXT), _meta/flow_truncated (BOOL), boolWidenedToJson (BOOL), dateValue (DATE), datetimeValue (TIMESTAMPTZ), flow_published_at (TIMESTAMPTZ), int64 (INT8), intWidenedToJson (INT8), multiple (JSON), nonScalarValue (JSON), numericString (NUMERIC), optional (JSON), requiredNumeric (NUMERIC), scalarValue (TEXT), stringWidenedToJson (TEXT), timeValue (TIME), flow_document (JSON) - -1, false, true, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.111111 +0000 GMT, 2024-09-13 01:01:01 +0000 +0000, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} +1, false, true, 2024-01-01T00:00:00Z, 2024-01-01T01:01:01.111111Z, 2024-09-13T01:01:01Z, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (TEXT), _meta/flow_truncated (BOOL), flow_published_at (TIMESTAMPTZ), int64 (INT8), multiple (JSON), nonScalarValue (JSON), optional (JSON), scalarValue (TEXT), flow_document (JSON), boolWidenedToJson (JSON), dateValue (TEXT), datetimeValue (TEXT), intWidenedToJson (JSON), numericString (TEXT), requiredNumeric (TEXT), stringWidenedToJson (JSON), timeValue (TEXT) - -1, false, 2024-09-13 01:01:01 +0000 +0000, 1, , , , test, {}, true, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, "hello", 01:01:01 +1, false, 2024-09-13T01:01:01Z, 1, , , , test, {}, true, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, "hello", 01:01:01 diff --git a/materialize-postgres/client.go b/materialize-postgres/client.go index dad794f0c4..36bcee916a 100644 --- a/materialize-postgres/client.go +++ b/materialize-postgres/client.go @@ -5,14 +5,13 @@ import ( stdsql "database/sql" "errors" "fmt" - "net" "strings" "time" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" pf "github.com/estuary/flow/go/protocols/flow" - "github.com/jackc/pgconn" + "github.com/jackc/pgx/v5/pgconn" log "github.com/sirupsen/logrus" _ "github.com/jackc/pgx/v5/stdlib" @@ -58,23 +57,14 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { if err := db.PingContext(ctx); err != nil { // Provide a more user-friendly representation of some common error causes. - var pgErr *pgconn.PgError - var netConnErr *net.DNSError - var netOpErr *net.OpError - + var pgErr *pgconn.ConnectError if errors.As(err, &pgErr) { - switch pgErr.Code { - case "28P01": + err = pgErr.Unwrap() + if errStr := err.Error(); strings.Contains(errStr, "(SQLSTATE 28P01)") { err = fmt.Errorf("incorrect username or password") - case "3D000": + } else if strings.Contains(errStr, "(SQLSTATE 3D000") { err = fmt.Errorf("database %q does not exist", cfg.Database) - } - } else if errors.As(err, &netConnErr) { - if netConnErr.IsNotFound { - err = fmt.Errorf("host at address %q cannot be found", cfg.Address) - } - } else if errors.As(err, &netOpErr) { - if netOpErr.Timeout() { + } else if strings.Contains(errStr, "context deadline exceeded") { err = fmt.Errorf("connection to host at address %q timed out (incorrect host or port?)", cfg.Address) } } @@ -98,11 +88,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return sql.StdFetchInfoSchema(ctx, c.db, pgDialect, catalog, resourcePaths) } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { var res tableConfig if tc.ResourceConfigJson != nil { @@ -189,10 +174,6 @@ func (c *client) CreateSchema(ctx context.Context, schemaName string) error { return sql.StdCreateSchema(ctx, c.db, pgDialect, schemaName) } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (string, string, error) { - return sql.StdFetchSpecAndVersion(ctx, c.db, specs, materialization) -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-postgres/driver.go b/materialize-postgres/driver.go index da1f691a14..d8e24e4328 100644 --- a/materialize-postgres/driver.go +++ b/materialize-postgres/driver.go @@ -225,13 +225,11 @@ func newPostgresDriver() *sql.Driver { if cfg.Schema != "" { metaBase = append(metaBase, cfg.Schema) } - var metaSpecs, metaCheckpoints = sql.MetaTables(metaBase) return &sql.Endpoint{ Config: cfg, Dialect: pgDialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable(metaBase), NewClient: newClient, CreateTableTemplate: tplCreateTargetTable, NewResource: newTableConfig, diff --git a/materialize-postgres/driver_test.go b/materialize-postgres/driver_test.go index 87e53b63ce..7f50776147 100644 --- a/materialize-postgres/driver_test.go +++ b/materialize-postgres/driver_test.go @@ -11,7 +11,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" "github.com/stretchr/testify/require" _ "github.com/jackc/pgx/v5/stdlib" @@ -54,16 +53,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", pgDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - pgDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - pgDialect.Literal(materialization.String()), - )) }, ) } @@ -122,16 +114,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", pgDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - pgDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - pgDialect.Literal(materialization.String()), - )) }, ) } @@ -198,14 +183,6 @@ func TestPrereqs(t *testing.T) { }, want: []error{fmt.Errorf("database %q does not exist", "wrong"+cfg.Database)}, }, - { - name: "wrong address", - cfg: func(cfg config) *config { - cfg.Address = "wrong." + cfg.Address - return &cfg - }, - want: []error{fmt.Errorf("host at address %q cannot be found", "wrong."+cfg.Address)}, - }, } for _, tt := range tests { diff --git a/materialize-redshift/.snapshots/TestValidateAndApply b/materialize-redshift/.snapshots/TestValidateAndApply index 6809a08b23..5ba1cb8d37 100644 --- a/materialize-redshift/.snapshots/TestValidateAndApply +++ b/materialize-redshift/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BOOLEAN' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'BIGINT' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'DOUBLE PRECISION' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'objField' is already being materialized as endpoint type 'SUPER' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP WITH TIME ZONE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'CHARACTER VARYING' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'DOUBLE PRECISION' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'NUMERIC' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-redshift/.snapshots/TestValidateAndApplyMigrations b/materialize-redshift/.snapshots/TestValidateAndApplyMigrations index 5355bb87ff..5934a3f784 100644 --- a/materialize-redshift/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-redshift/.snapshots/TestValidateAndApplyMigrations @@ -41,8 +41,7 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: key (VARCHAR), _meta/flow_truncated (BOOL), boolwidenedtojson (BOOL), datevalue (DATE), datetimevalue (TIMESTAMPTZ), flow_published_at (TIMESTAMPTZ), int64 (INT8), intwidenedtojson (INT8), multiple (4000), nonscalarvalue (4000), numericstring (NUMERIC), optional (4000), requirednumeric (NUMERIC), scalarvalue (VARCHAR), stringwidenedtojson (VARCHAR), timevalue (VARCHAR), flow_document (4000) - -1, false, true, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.111111 +0000 GMT, 2024-09-13 02:01:01 +0100 IST, 1, 999, , , 123, , 456, test, hello, 01:01:01, "{}" +1, false, true, 2024-01-01T00:00:00Z, 2024-01-01T01:01:01.111111Z, 2024-09-13T01:01:01Z, 1, 999, , , 123, , 456, test, hello, 01:01:01, "{}" Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: key (VARCHAR), _meta/flow_truncated (BOOL), boolwidenedtojson (TIMESTAMPTZ), datevalue (INT8), datetimevalue (4000), flow_published_at (4000), int64 (4000), intwidenedtojson (VARCHAR), multiple (VARCHAR), nonscalarvalue (4000), numericstring (4000), optional (VARCHAR), requirednumeric (VARCHAR), scalarvalue (4000), stringwidenedtojson (VARCHAR), timevalue (VARCHAR), flow_document (4000) - -1, false, 2024-09-13 02:01:01 +0100 IST, 1, , , , test, 01:01:01, "{}", true, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, "hello" +1, false, 2024-09-13T01:01:01Z, 1, , , , test, 01:01:01, "{}", true, 2024-01-01, 2024-01-01T01:01:01.111111Z, 999, 123, 456, "hello" diff --git a/materialize-redshift/client.go b/materialize-redshift/client.go index f4a0ce71e5..6fcbbd853f 100644 --- a/materialize-redshift/client.go +++ b/materialize-redshift/client.go @@ -10,7 +10,6 @@ import ( "errors" "fmt" "io" - "net" "net/http" "path" "strings" @@ -21,10 +20,9 @@ import ( awsHttp "github.com/aws/smithy-go/transport/http" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" "github.com/google/uuid" - "github.com/jackc/pgconn" "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgconn" log "github.com/sirupsen/logrus" ) @@ -64,35 +62,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return sql.StdFetchInfoSchema(ctx, c.db, c.ep.Dialect, catalog, resourcePaths) } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - // Compress the spec bytes to store, since we are using a VARBYTE column with a limit of 1MB. - // TODO(whb): This will go away when we start passing in the last validated spec to Validate - // calls and stop needing to persist a spec at all. - // updateSpec.Parameters - specB64 := updateSpec.Parameters[1] // The second parameter is the spec - specBytes, err := base64.StdEncoding.DecodeString(specB64.(string)) - if err != nil { - return fmt.Errorf("decoding base64 spec prior to compressing: %w", err) - } - - // Sanity check that this is indeed a spec. By all rights this is totally unnecessary but it - // makes me feel a little better about indexing updateSpec.Parameters up above. - var spec pf.MaterializationSpec - if err := spec.Unmarshal(specBytes); err != nil { - return fmt.Errorf("application logic error - specBytes was not a spec: %w", err) - } - - compressed, err := compressBytes(specBytes) - if err != nil { - return fmt.Errorf("compressing spec bytes: %w", err) - } - - updateSpec.Parameters[1] = base64.StdEncoding.EncodeToString(compressed) - - _, err = c.db.ExecContext(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { _, err := c.db.ExecContext(ctx, tc.TableCreateSql) return err @@ -189,28 +158,20 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { if err := db.PingContext(pingCtx); err != nil { // Provide a more user-friendly representation of some common error causes. - var pgErr *pgconn.PgError - var netConnErr *net.DNSError - var netOpErr *net.OpError + var pgErr *pgconn.ConnectError if errors.As(err, &pgErr) { - switch pgErr.Code { - case "28000": + err = pgErr.Unwrap() + if errStr := err.Error(); strings.Contains(errStr, "(SQLSTATE 28000)") { err = fmt.Errorf("incorrect username or password") - case "3D000": + } else if strings.Contains(errStr, "(SQLSTATE 3D000") { err = fmt.Errorf("database %q does not exist", cfg.Database) - } - } else if errors.As(err, &netConnErr) { - if netConnErr.IsNotFound { - err = fmt.Errorf("host at address %q cannot be found", cfg.Address) - } - } else if errors.As(err, &netOpErr) { - if netOpErr.Timeout() { + } else if strings.Contains(errStr, "context deadline exceeded") { errStr := `connection to host at address %q timed out, possible causes: - * Redshift endpoint is not set to be publicly accessible - * there is no inbound rule allowing Estuary's IP address to connect through the Redshift VPC security group - * the configured address is incorrect, possibly with an incorrect host or port - * if connecting through an SSH tunnel, the SSH bastion server may not be operational, or the connection details are incorrect` + * Redshift endpoint is not set to be publicly accessible + * there is no inbound rule allowing Estuary's IP address to connect through the Redshift VPC security group + * the configured address is incorrect, possibly with an incorrect host or port + * if connecting through an SSH tunnel, the SSH bastion server may not be operational, or the connection details are incorrect` err = fmt.Errorf(errStr, cfg.Address) } } @@ -266,32 +227,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (string, string, error) { - var version, spec string - - if err := c.db.QueryRowContext( - ctx, - fmt.Sprintf( - "SELECT version, spec FROM %s WHERE materialization = %s;", - specs.Identifier, - specs.Keys[0].Placeholder, - ), - materialization.String(), - ).Scan(&version, &spec); err != nil { - return "", "", err - } - - if hexBytes, err := hex.DecodeString(spec); err != nil { - return "", "", fmt.Errorf("hex.DecodeString: %w", err) - } else if specBytes, err := base64.StdEncoding.DecodeString(string(hexBytes)); err != nil { - return "", "", fmt.Errorf("base64.DecodeString: %w", err) - } else if specBytes, err = maybeDecompressBytes(specBytes); err != nil { - return "", "", fmt.Errorf("decompressing spec: %w", err) - } else { - return base64.StdEncoding.EncodeToString(specBytes), version, nil - } -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return c.withDB(func(db *stdsql.DB) error { return sql.StdSQLExecStatements(ctx, db, statements) }) } diff --git a/materialize-redshift/driver.go b/materialize-redshift/driver.go index 00e0f503b9..0f5b552158 100644 --- a/materialize-redshift/driver.go +++ b/materialize-redshift/driver.go @@ -232,7 +232,6 @@ func newRedshiftDriver() *sql.Driver { if cfg.Schema != "" { metaBase = append(metaBase, cfg.Schema) } - metaSpecs, metaCheckpoints := sql.MetaTables(metaBase) db, err := stdsql.Open("pgx", cfg.toURI()) if err != nil { @@ -256,8 +255,7 @@ func newRedshiftDriver() *sql.Driver { return &sql.Endpoint{ Config: cfg, Dialect: dialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable(metaBase), NewClient: newClient, CreateTableTemplate: templates.createTargetTable, NewResource: newTableConfig, diff --git a/materialize-redshift/driver_test.go b/materialize-redshift/driver_test.go index e4447172fa..05be370ed8 100644 --- a/materialize-redshift/driver_test.go +++ b/materialize-redshift/driver_test.go @@ -13,7 +13,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/google/uuid" "github.com/jackc/pgx/v5" @@ -79,16 +78,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -147,16 +139,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -242,10 +227,6 @@ func TestFencingCases(t *testing.T) { } func TestPrereqs(t *testing.T) { - // These tests assume that the configuration obtained from environment variables forms a valid - // config that could be used to materialize into Redshift. Various parameters of the - // configuration are then manipulated to test assertions for incorrect configs. - cfg := mustGetCfg(t) nonExistentBucket := uuid.NewString() @@ -284,14 +265,6 @@ func TestPrereqs(t *testing.T) { }, want: []error{fmt.Errorf("database %q does not exist", "wrong"+cfg.Database)}, }, - { - name: "wrong address", - cfg: func(cfg config) *config { - cfg.Address = "wrong." + cfg.Address - return &cfg - }, - want: []error{fmt.Errorf("host at address %q cannot be found", "wrong."+cfg.Address)}, - }, { name: "bucket doesn't exist", cfg: func(cfg config) *config { diff --git a/materialize-s3-iceberg/.snapshots/TestSpec b/materialize-s3-iceberg/.snapshots/TestSpec index f7664d1a0c..4b6a530139 100644 --- a/materialize-s3-iceberg/.snapshots/TestSpec +++ b/materialize-s3-iceberg/.snapshots/TestSpec @@ -6,9 +6,6 @@ "catalog_type": { "const": "AWS Glue", "default": "AWS Glue", - "enum": [ - "AWS Glue" - ], "title": "Catalog Type", "type": "string" } @@ -21,9 +18,6 @@ "catalog_type": { "const": "Iceberg REST Server", "default": "Iceberg REST Server", - "enum": [ - "Iceberg REST Server" - ], "order": 0, "title": "Catalog Type", "type": "string" @@ -120,7 +114,7 @@ "type": "string" }, "upload_interval": { - "default": "PT5M", + "default": "PT300S", "description": "Frequency at which files will be uploaded. Must be a valid ISO8601 duration string no greater than 4 hours.", "format": "duration", "order": 6, diff --git a/materialize-s3-iceberg/catalog.go b/materialize-s3-iceberg/catalog.go index a3693175ef..c634cf6729 100644 --- a/materialize-s3-iceberg/catalog.go +++ b/materialize-s3-iceberg/catalog.go @@ -12,20 +12,14 @@ import ( ) type catalog struct { - cfg *config - // TODO(whb): Including the lastSpec from the validate or apply request is a temporary hack - // until we get around to removing the "load/persist a spec in the destination" concept more - // thoroughly. As of this writing, the iceberg materialization is the first one to actually use - // the lastSpec from the validate or apply request. - lastSpec *pf.MaterializationSpec + cfg *config resourcePaths [][]string } -func newCatalog(cfg config, resourcePaths [][]string, lastSpec *pf.MaterializationSpec) *catalog { +func newCatalog(cfg config, resourcePaths [][]string) *catalog { return &catalog{ cfg: &cfg, resourcePaths: resourcePaths, - lastSpec: lastSpec, } } @@ -244,17 +238,3 @@ func (c *catalog) appendFiles( return nil } - -// These functions are vestigial from the age of persisting specs in the destination. - -func (c *catalog) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, boilerplate.ActionApplyFn, error) { - return "", nil, nil -} - -func (c *catalog) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - return c.lastSpec, nil -} - -func (c *catalog) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, exists bool) (string, boilerplate.ActionApplyFn, error) { - return "", nil, nil -} diff --git a/materialize-s3-iceberg/driver.go b/materialize-s3-iceberg/driver.go index f4fbe01c52..6a8b1f941a 100644 --- a/materialize-s3-iceberg/driver.go +++ b/materialize-s3-iceberg/driver.go @@ -256,7 +256,7 @@ func (driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Respo resourcePaths = append(resourcePaths, res.path()) } - catalog := newCatalog(cfg, resourcePaths, req.LastMaterialization) + catalog := newCatalog(cfg, resourcePaths) is, err := catalog.infoSchema(ctx) if err != nil { @@ -304,7 +304,7 @@ func (driver) Apply(ctx context.Context, req *pm.Request_Apply) (*pm.Response_Ap resourcePaths = append(resourcePaths, b.ResourcePath) } - catalog := newCatalog(cfg, resourcePaths, req.LastMaterialization) + catalog := newCatalog(cfg, resourcePaths) existingNamespaces, err := catalog.listNamespaces(ctx) if err != nil { @@ -361,7 +361,7 @@ func (d driver) NewTransactor(ctx context.Context, open pm.Request_Open, _ *boil }) } - catalog := newCatalog(cfg, resourcePaths, open.Materialization) + catalog := newCatalog(cfg, resourcePaths) tablePaths, err := catalog.tablePaths(ctx, resourcePaths) if err != nil { return nil, nil, nil, fmt.Errorf("looking up table paths: %w", err) diff --git a/materialize-s3-iceberg/driver_test.go b/materialize-s3-iceberg/driver_test.go index aa3c8bd6ee..1d90db1286 100644 --- a/materialize-s3-iceberg/driver_test.go +++ b/materialize-s3-iceberg/driver_test.go @@ -10,7 +10,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/stretchr/testify/require" ) @@ -94,9 +93,8 @@ func TestValidateAndApply(t *testing.T) { return out.String() }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, do, _ := catalog.DeleteResource(ctx, resourceConfig.path()) do(ctx) }, diff --git a/materialize-snowflake/.snapshots/TestSpecification b/materialize-snowflake/.snapshots/TestSpecification index c9b28a491d..6baa3c5ee0 100644 --- a/materialize-snowflake/.snapshots/TestSpecification +++ b/materialize-snowflake/.snapshots/TestSpecification @@ -19,7 +19,7 @@ "schema": { "type": "string", "title": "Schema", - "description": "Database schema for bound collection tables (unless overridden within the binding resource configuration) as well as associated materialization metadata tables.", + "description": "Database schema for bound collection tables (unless overridden within the binding resource configuration).", "order": 4 }, "warehouse": { diff --git a/materialize-snowflake/.snapshots/TestValidateAndApply b/materialize-snowflake/.snapshots/TestValidateAndApply index 229c7f0d0e..19c60fe6d8 100644 --- a/materialize-snowflake/.snapshots/TestValidateAndApply +++ b/materialize-snowflake/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BOOLEAN' but endpoint type 'INTEGER' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'NUMBER' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'BOOLEAN' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'objField' is already being materialized as endpoint type 'VARIANT' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'TIMESTAMP_LTZ' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'TEXT' but endpoint type 'INTEGER' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'NUMBER' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'NUMBER' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'NUMBER' but endpoint type 'TEXT' is required by its schema '{ type: [string] }'"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-snowflake/.snapshots/TestValidateAndApplyMigrations b/materialize-snowflake/.snapshots/TestValidateAndApplyMigrations index f77b6b0f48..db994e429d 100644 --- a/materialize-snowflake/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-snowflake/.snapshots/TestValidateAndApplyMigrations @@ -41,8 +41,7 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: KEY (TEXT), _meta/flow_truncated (BOOLEAN), BOOLWIDENEDTOJSON (BOOLEAN), DATEVALUE (DATE), DATETIMEVALUE (TIMESTAMP_LTZ), FLOW_PUBLISHED_AT (TIMESTAMP_LTZ), INT64 (FIXED), INTWIDENEDTOJSON (FIXED), MULTIPLE (VARIANT), NONSCALARVALUE (VARIANT), NUMERICSTRING (FIXED), OPTIONAL (VARIANT), REQUIREDNUMERIC (FIXED), SCALARVALUE (TEXT), STRINGWIDENEDTOJSON (TEXT), TIMEVALUE (TEXT), FLOW_DOCUMENT (VARIANT) - -1, 0, 1, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.111111111 -0800 PST, 2024-09-13 01:01:01 -0700 PDT, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} +1, 0, 1, 2024-01-01T00:00:00Z, 2024-01-01T09:01:01.111111111Z, 2024-09-13T08:01:01Z, 1, 999, , , 123, , 456, test, hello, 01:01:01, {} Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -87,6 +86,5 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: KEY (TEXT), _meta/flow_truncated (BOOLEAN), FLOW_PUBLISHED_AT (TIMESTAMP_LTZ), INT64 (FIXED), MULTIPLE (VARIANT), NONSCALARVALUE (VARIANT), OPTIONAL (VARIANT), SCALARVALUE (TEXT), TIMEVALUE (TEXT), FLOW_DOCUMENT (VARIANT), BOOLWIDENEDTOJSON (VARIANT), DATEVALUE (TEXT), DATETIMEVALUE (TEXT), INTWIDENEDTOJSON (VARIANT), NUMERICSTRING (TEXT), REQUIREDNUMERIC (TEXT), STRINGWIDENEDTOJSON (VARIANT) - -1, 0, 2024-09-13 01:01:01 -0700 PDT, 1, , , , test, 01:01:01, {}, true, 2024-01-01, 2024-01-01T09:01:01.111111111Z, 999, 123, 456, "hello" +1, 0, 2024-09-13T08:01:01Z, 1, , , , test, 01:01:01, {}, true, 2024-01-01, 2024-01-01T09:01:01.111111111Z, 999, 123, 456, "hello" diff --git a/materialize-snowflake/client.go b/materialize-snowflake/client.go index dce4e39c0b..d84e15c66b 100644 --- a/materialize-snowflake/client.go +++ b/materialize-snowflake/client.go @@ -10,7 +10,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" sf "github.com/snowflakedb/gosnowflake" ) @@ -58,11 +57,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (is * return sql.StdFetchInfoSchema(ctx, c.db, c.ep.Dialect, catalog, resourcePaths) } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - // The error message returned from Snowflake for the multi-statement table // creation queries is mostly a bunch of garbled nonsense about Javascript // execution errors if the user doesn't have permission to create tables in the @@ -150,13 +144,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { var sfError *sf.SnowflakeError if errors.As(err, &sfError) { switch sfError.Number { - case 260008: - // This is the error if the host URL has an incorrect account identifier. The error - // message from the Snowflake driver will accurately report that the account name is - // incorrect, but would be confusing for a user because we have a separate "Account" - // input field. We want to be specific here and report that it is the account - // identifier in the host URL. - err = fmt.Errorf("incorrect account identifier %q in host URL", strings.TrimSuffix(cfg.Host, ".snowflakecomputing.com")) case 390100: err = fmt.Errorf("incorrect username or password") case 390201: @@ -186,10 +173,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (string, string, error) { - return sql.StdFetchSpecAndVersion(ctx, c.db, specs, materialization) -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-snowflake/config.go b/materialize-snowflake/config.go index 8884a7afa4..56693bfa0e 100644 --- a/materialize-snowflake/config.go +++ b/materialize-snowflake/config.go @@ -23,7 +23,7 @@ import ( type config struct { Host string `json:"host" jsonschema:"title=Host (Account URL),description=The Snowflake Host used for the connection. Must include the account identifier and end in .snowflakecomputing.com. Example: orgname-accountname.snowflakecomputing.com (do not include the protocol)." jsonschema_extras:"order=0,pattern=^[^/:]+.snowflakecomputing.com$"` Database string `json:"database" jsonschema:"title=Database,description=The SQL database to connect to." jsonschema_extras:"order=3"` - Schema string `json:"schema" jsonschema:"title=Schema,description=Database schema for bound collection tables (unless overridden within the binding resource configuration) as well as associated materialization metadata tables." jsonschema_extras:"order=4"` + Schema string `json:"schema" jsonschema:"title=Schema,description=Database schema for bound collection tables (unless overridden within the binding resource configuration)." jsonschema_extras:"order=4"` Warehouse string `json:"warehouse,omitempty" jsonschema:"title=Warehouse,description=The Snowflake virtual warehouse used to execute queries. Uses the default warehouse for the Snowflake user if left blank." jsonschema_extras:"order=5"` Role string `json:"role,omitempty" jsonschema:"title=Role,description=The user role used to perform actions." jsonschema_extras:"order=6"` Account string `json:"account,omitempty" jsonschema:"title=Account,description=Optional Snowflake account identifier." jsonschema_extras:"order=7"` diff --git a/materialize-snowflake/snowflake.go b/materialize-snowflake/snowflake.go index b00757bc7b..280c840a21 100644 --- a/materialize-snowflake/snowflake.go +++ b/materialize-snowflake/snowflake.go @@ -97,9 +97,6 @@ func newSnowflakeDriver() *sql.Driver { "tenant": tenant, }).Info("opening Snowflake") - var metaBase sql.TablePath = []string{parsed.Schema} - var metaSpecs, _ = sql.MetaTables(metaBase) - dsn, err := parsed.toURI(tenant) if err != nil { return nil, fmt.Errorf("building snowflake dsn: %w", err) @@ -120,9 +117,8 @@ func newSnowflakeDriver() *sql.Driver { var templates = renderTemplates(dialect) return &sql.Endpoint{ - Config: parsed, - Dialect: dialect, - MetaSpecs: &metaSpecs, + Config: parsed, + Dialect: dialect, // Snowflake does not use the checkpoint table, instead we use the recovery log // as the authoritative checkpoint and idempotent apply pattern MetaCheckpoints: nil, diff --git a/materialize-snowflake/snowflake_test.go b/materialize-snowflake/snowflake_test.go index 72c57bb0f4..72768e61ae 100644 --- a/materialize-snowflake/snowflake_test.go +++ b/materialize-snowflake/snowflake_test.go @@ -12,7 +12,6 @@ import ( "github.com/bradleyjkemp/cupaloy" boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" pm "github.com/estuary/flow/go/protocols/materialize" "github.com/stretchr/testify/require" @@ -82,16 +81,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -153,16 +145,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -179,10 +164,6 @@ func TestSpecification(t *testing.T) { } func TestPrereqs(t *testing.T) { - // These tests assume that the configuration obtained from environment variables forms a valid - // config that could be used to materialize into Snowflake. Various parameters of the - // configuration are then manipulated to test assertions for incorrect configs. - cfg := mustGetCfg(t) tests := []struct { @@ -195,14 +176,6 @@ func TestPrereqs(t *testing.T) { cfg: func(cfg config) *config { return &cfg }, want: nil, }, - { - name: "wrong account identifier in host", - cfg: func(cfg config) *config { - cfg.Host = "wrong.snowflakecomputing.com" - return &cfg - }, - want: []error{fmt.Errorf("incorrect account identifier %q in host URL", "wrong")}, - }, { name: "wrong username", cfg: func(cfg config) *config { diff --git a/materialize-sql/.snapshots/TestTableTemplate b/materialize-sql/.snapshots/TestTableTemplate index 17cc58b6f6..ea6beca43c 100644 --- a/materialize-sql/.snapshots/TestTableTemplate +++ b/materialize-sql/.snapshots/TestTableTemplate @@ -1,5 +1,5 @@ - CREATE TABLE one."reserved".checkpoints ( + CREATE TABLE one."reserved".flow_checkpoints_v1 ( materialization TEXT NOT NULL, key_begin BIGINT NOT NULL, key_end BIGINT NOT NULL, @@ -9,10 +9,10 @@ ); - COMMENT ON TABLE one."reserved".checkpoints IS 'This table holds Flow processing checkpoints used for exactly-once processing of materializations'; - COMMENT ON COLUMN one."reserved".checkpoints.materialization IS 'The name of the materialization.'; - COMMENT ON COLUMN one."reserved".checkpoints.key_begin IS 'The inclusive lower-bound key hash covered by this checkpoint.'; - COMMENT ON COLUMN one."reserved".checkpoints.key_end IS 'The inclusive upper-bound key hash covered by this checkpoint.'; - COMMENT ON COLUMN one."reserved".checkpoints.fence IS 'This nonce is used to uniquely identify unique process assignments of a shard and prevent them from conflicting.'; - COMMENT ON COLUMN one."reserved".checkpoints.checkpoint IS 'Checkpoint of the Flow consumer shard, encoded as base64 protobuf.'; + COMMENT ON TABLE one."reserved".flow_checkpoints_v1 IS 'This table holds Flow processing checkpoints used for exactly-once processing of materializations'; + COMMENT ON COLUMN one."reserved".flow_checkpoints_v1.materialization IS 'The name of the materialization.'; + COMMENT ON COLUMN one."reserved".flow_checkpoints_v1.key_begin IS 'The inclusive lower-bound key hash covered by this checkpoint.'; + COMMENT ON COLUMN one."reserved".flow_checkpoints_v1.key_end IS 'The inclusive upper-bound key hash covered by this checkpoint.'; + COMMENT ON COLUMN one."reserved".flow_checkpoints_v1.fence IS 'This nonce is used to uniquely identify unique process assignments of a shard and prevent them from conflicting.'; + COMMENT ON COLUMN one."reserved".flow_checkpoints_v1.checkpoint IS 'Checkpoint of the Flow consumer shard, encoded as base64 protobuf.'; diff --git a/materialize-sql/apply.go b/materialize-sql/apply.go index 682ef1f733..cbd5a4c26c 100644 --- a/materialize-sql/apply.go +++ b/materialize-sql/apply.go @@ -2,12 +2,9 @@ package sql import ( "context" - "database/sql" - "encoding/base64" "encoding/json" "fmt" "slices" - "strings" boilerplate "github.com/estuary/connectors/materialize-boilerplate" pf "github.com/estuary/flow/go/protocols/flow" @@ -41,14 +38,6 @@ type TableAlter struct { ColumnTypeChanges []ColumnTypeMigration } -// MetaSpecsUpdate is an endpoint-specific parameterized query and parameters needed to persist a -// new or updated materialization spec. -type MetaSpecsUpdate struct { - ParameterizedQuery string - Parameters []interface{} - QueryString string // For endpoints that do not support parameterized queries. -} - var _ boilerplate.Applier = (*sqlApplier)(nil) type sqlApplier struct { @@ -67,51 +56,6 @@ func newSqlApplier(client Client, is *boilerplate.InfoSchema, endpoint *Endpoint } } -func (a *sqlApplier) CreateMetaTables(ctx context.Context, spec *pf.MaterializationSpec) (string, boilerplate.ActionApplyFn, error) { - if (a.endpoint.MetaCheckpoints == nil || a.is.HasResource(a.endpoint.MetaCheckpoints.Path)) && - (a.endpoint.MetaSpecs == nil || a.is.HasResource(a.endpoint.MetaSpecs.Path)) { - // If this materialization does not use the checkpoints or specs table OR it does and - // they already exist, there is nothing more to do here. - return "", nil, nil - } - - var creates []TableCreate - var actionDesc []string - - for _, meta := range []*TableShape{a.endpoint.MetaSpecs, a.endpoint.MetaCheckpoints} { - if meta == nil { - continue - } - resolved, err := ResolveTable(*meta, a.endpoint.Dialect) - if err != nil { - return "", nil, err - } - createStatement, err := RenderTableTemplate(resolved, a.endpoint.CreateTableTemplate) - if err != nil { - return "", nil, err - } - creates = append(creates, TableCreate{ - Table: resolved, - TableCreateSql: createStatement, - ResourceConfigJson: nil, // not applicable for meta tables - }) - actionDesc = append(actionDesc, createStatement) - } - - if len(creates) == 0 { - return "", nil, nil - } - - return strings.Join(actionDesc, "\n"), func(ctx context.Context) error { - for _, c := range creates { - if err := a.client.CreateTable(ctx, c); err != nil { - return fmt.Errorf("creating meta table %s: %w", c.Identifier, err) - } - } - return nil - }, nil -} - func (a *sqlApplier) CreateResource(ctx context.Context, spec *pf.MaterializationSpec, bindingIndex int) (string, boilerplate.ActionApplyFn, error) { table, err := getTable(a.endpoint, spec, bindingIndex) if err != nil { @@ -140,69 +84,6 @@ func (a *sqlApplier) CreateResource(ctx context.Context, spec *pf.Materializatio }, nil } -func (a *sqlApplier) LoadSpec(ctx context.Context, materialization pf.Materialization) (*pf.MaterializationSpec, error) { - spec, _, err := loadSpec(ctx, a.client, a.endpoint, materialization) - if err != nil { - return nil, err - } - - return spec, nil -} - -func (a *sqlApplier) PutSpec(ctx context.Context, spec *pf.MaterializationSpec, version string, exists bool) (string, boilerplate.ActionApplyFn, error) { - specBytes, err := spec.Marshal() - if err != nil { - panic(err) // Cannot fail to marshal. - } - - var description string - var specUpdate MetaSpecsUpdate - if a.endpoint.MetaSpecs != nil { - // Insert or update the materialization specification. Both parameterized queries and - // literal query strings are supported. A parameterized query is generally preferable, but - // some endpoints don't have support for those. - var paramArgs = []interface{}{ - a.endpoint.Identifier(a.endpoint.MetaSpecs.Path...), - a.endpoint.Placeholder(0), - a.endpoint.Placeholder(1), - a.endpoint.Placeholder(2), - } - var params = []interface{}{ - version, - base64.StdEncoding.EncodeToString(specBytes), - spec.Name.String(), - } - var queryStringArgs = []interface{}{ - paramArgs[0], - a.endpoint.Literal(params[0].(string)), - a.endpoint.Literal(params[1].(string)), - a.endpoint.Literal(params[2].(string)), - } - var descriptionArgs = []interface{}{ - paramArgs[0], - a.endpoint.Literal(params[0].(string)), - a.endpoint.Literal("(a-base64-encoded-value)"), - a.endpoint.Literal(params[2].(string)), - } - - var q string - if exists { - q = "UPDATE %[1]s SET version = %[2]s, spec = %[3]s WHERE materialization = %[4]s;" - } else { - q = "INSERT INTO %[1]s (version, spec, materialization) VALUES (%[2]s, %[3]s, %[4]s);" - } - - specUpdate.ParameterizedQuery = fmt.Sprintf(q, paramArgs...) - specUpdate.Parameters = params - specUpdate.QueryString = fmt.Sprintf(q, queryStringArgs...) - description = fmt.Sprintf(q, descriptionArgs...) - } - - return description, func(ctx context.Context) error { - return a.client.PutSpec(ctx, specUpdate) - }, nil -} - func (a *sqlApplier) DeleteResource(ctx context.Context, path []string) (string, boilerplate.ActionApplyFn, error) { return a.client.DeleteTable(ctx, path) } @@ -317,40 +198,3 @@ func getTable(endpoint *Endpoint, spec *pf.MaterializationSpec, bindingIndex int tableShape := BuildTableShape(spec, bindingIndex, resource) return ResolveTable(tableShape, endpoint.Dialect) } - -// loadSpec loads the named MaterializationSpec and its version that's stored within the Endpoint, -// if any. -func loadSpec(ctx context.Context, client Client, endpoint *Endpoint, materialization pf.Materialization) (*pf.MaterializationSpec, string, error) { - var ( - err error - metaSpecs Table - spec = new(pf.MaterializationSpec) - specB64, version string - ) - - if endpoint.MetaSpecs == nil { - return nil, "", nil - } - if metaSpecs, err = ResolveTable(*endpoint.MetaSpecs, endpoint.Dialect); err != nil { - return nil, "", fmt.Errorf("resolving specifications table: %w", err) - } - specB64, version, err = client.FetchSpecAndVersion(ctx, metaSpecs, materialization) - - if err == sql.ErrNoRows { - return nil, "", nil - } else if err != nil { - log.WithFields(log.Fields{ - "table": endpoint.MetaSpecs.Path, - "err": err, - }).Info("failed to query materialization spec (the table may not be initialized?)") - return nil, "", nil - } else if specBytes, err := base64.StdEncoding.DecodeString(specB64); err != nil { - return nil, version, fmt.Errorf("base64.Decode: %w", err) - } else if err = spec.Unmarshal(specBytes); err != nil { - return nil, version, fmt.Errorf("spec.Unmarshal: %w", err) - } else if err = spec.Validate(); err != nil { - return nil, version, fmt.Errorf("validating spec: %w", err) - } - - return spec, version, nil -} diff --git a/materialize-sql/driver.go b/materialize-sql/driver.go index de78b9e190..0069aa024a 100644 --- a/materialize-sql/driver.go +++ b/materialize-sql/driver.go @@ -72,12 +72,11 @@ func (d *Driver) Spec(ctx context.Context, req *pm.Request_Spec) (*pm.Response_S func (d *Driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Response_Validated, error) { var ( - err error - endpoint *Endpoint - client Client - loadedSpec *pf.MaterializationSpec - resp = new(pm.Response_Validated) - conf = d.EndpointSpecType + err error + endpoint *Endpoint + client Client + resp = new(pm.Response_Validated) + conf = d.EndpointSpecType ) if err = req.Validate(); err != nil { @@ -95,10 +94,6 @@ func (d *Driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Re } defer client.Close() - if loadedSpec, _, err = loadSpec(ctx, client, endpoint, req.Name); err != nil { - return nil, fmt.Errorf("loading current applied materialization spec: %w", err) - } - resources := make([]Resource, 0, len(req.Bindings)) resourcePaths := make([][]string, 0, len(req.Bindings)) for _, b := range req.Bindings { @@ -109,9 +104,6 @@ func (d *Driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Re resources = append(resources, res) resourcePaths = append(resourcePaths, res.Path()) } - if endpoint.MetaSpecs != nil { - resourcePaths = append(resourcePaths, endpoint.MetaSpecs.Path) - } if endpoint.MetaCheckpoints != nil { resourcePaths = append(resourcePaths, endpoint.MetaCheckpoints.Path) } @@ -146,7 +138,7 @@ func (d *Driver) Validate(ctx context.Context, req *pm.Request_Validate) (*pm.Re bindingSpec.Backfill, bindingSpec.Collection, bindingSpec.FieldConfigJsonMap, - loadedSpec, + req.LastMaterialization, ) if err != nil { return nil, err @@ -188,9 +180,6 @@ func (d *Driver) Apply(ctx context.Context, req *pm.Request_Apply) (*pm.Response for _, b := range req.Materialization.Bindings { resourcePaths = append(resourcePaths, b.ResourcePath) } - if endpoint.MetaSpecs != nil { - resourcePaths = append(resourcePaths, endpoint.MetaSpecs.Path) - } if endpoint.MetaCheckpoints != nil { resourcePaths = append(resourcePaths, endpoint.MetaCheckpoints.Path) } @@ -222,11 +211,27 @@ func (d *Driver) Apply(ctx context.Context, req *pm.Request_Apply) (*pm.Response } } + if endpoint.MetaCheckpoints != nil && !is.HasResource(endpoint.MetaCheckpoints.Path) { + // Create the checkpoints table if it doesn't already exist. + if resolved, err := ResolveTable(*endpoint.MetaCheckpoints, endpoint.Dialect); err != nil { + return nil, err + } else if createStatement, err := RenderTableTemplate(resolved, endpoint.CreateTableTemplate); err != nil { + return nil, err + } else if err := client.CreateTable(ctx, TableCreate{ + Table: resolved, + TableCreateSql: createStatement, + ResourceConfigJson: nil, // not applicable for meta tables + }); err != nil { + return nil, fmt.Errorf("creating checkpoints table: %w", err) + } else { + log.WithField("table", resolved.Identifier).Info("created checkpoints table") + } + } + return boilerplate.ApplyChanges(ctx, req, newSqlApplier(client, is, endpoint, constrainter{dialect: endpoint.Dialect}), is, endpoint.ConcurrentApply) } func (d *Driver) NewTransactor(ctx context.Context, open pm.Request_Open, be *boilerplate.BindingEvents) (m.Transactor, *pm.Response_Opened, *boilerplate.MaterializeOptions, error) { - var loadedVersion string var conf = d.EndpointSpecType if err := json.Unmarshal(open.Materialization.ConfigJson, conf); err != nil { @@ -247,20 +252,6 @@ func (d *Driver) NewTransactor(ctx context.Context, open pm.Request_Open, be *bo defer client.Close() var resourcePaths [][]string - if endpoint.MetaSpecs != nil { - resourcePaths = append(resourcePaths, endpoint.MetaSpecs.Path) - - if _, loadedVersion, err = loadSpec(ctx, client, endpoint, open.Materialization.Name); err != nil { - return nil, nil, nil, fmt.Errorf("loading prior applied materialization spec: %w", err) - } else if loadedVersion == "" { - return nil, nil, nil, fmt.Errorf("materialization has not been applied") - } else if loadedVersion != open.Version { - return nil, nil, nil, fmt.Errorf( - "applied and current materializations are different versions (applied: %s vs current: %s)", - loadedVersion, open.Version) - } - } - var tables []Table for index, spec := range open.Materialization.Bindings { resourcePaths = append(resourcePaths, spec.ResourcePath) diff --git a/materialize-sql/endpoint.go b/materialize-sql/endpoint.go index e3da9acd20..13d6dc944a 100644 --- a/materialize-sql/endpoint.go +++ b/materialize-sql/endpoint.go @@ -13,9 +13,6 @@ import ( ) type Client interface { - // FetchSpecAndVersion retrieves the materialization from Table `specs`, - // or returns sql.ErrNoRows if no such spec exists. - FetchSpecAndVersion(ctx context.Context, specs Table, materialization pf.Materialization) (specB64, version string, _ error) ExecStatements(ctx context.Context, statements []string) error InstallFence(ctx context.Context, checkpoints Table, fence Fence) (Fence, error) @@ -25,9 +22,6 @@ type Client interface { // the relevant schemas. InfoSchema(ctx context.Context, resourcePaths [][]string) (*boilerplate.InfoSchema, error) - // PutSpec executes the MetaSpecsUpdate to upsert the spec into the metadata table. - PutSpec(ctx context.Context, spec MetaSpecsUpdate) error - // CreateTable creates a table in the destination system. CreateTable(ctx context.Context, tc TableCreate) error @@ -97,8 +91,6 @@ type Endpoint struct { Config interface{} // Dialect of the Endpoint. Dialect - // MetaSpecs is the specification meta-table of the Endpoint. - MetaSpecs *TableShape // MetaCheckpoints is the checkpoints meta-table of the Endpoint. // It's optional, and won't be created or used if it's nil. MetaCheckpoints *TableShape diff --git a/materialize-sql/meta_tables.go b/materialize-sql/meta_tables.go index 13e34dc165..39f40c8491 100644 --- a/materialize-sql/meta_tables.go +++ b/materialize-sql/meta_tables.go @@ -7,22 +7,13 @@ import ( const ( // DefaultFlowCheckpoints is the default table for checkpoints. DefaultFlowCheckpoints = "flow_checkpoints_v1" - // DefaultFlowMaterializations is the default table for materialization specs. - DefaultFlowMaterializations = "flow_materializations_v2" ) -// MetaTables returns *Table configurations for Flow materialization metadata tables, -// having the given `base` TablePath extended with their well-known table names. -func MetaTables(base TablePath) (specs, checkpoints TableShape) { - return FlowMaterializationsTable(append(base, DefaultFlowMaterializations)...), - FlowCheckpointsTable(append(base, DefaultFlowCheckpoints)...) -} - // FlowCheckpointsTable returns the TableShape that (optionally) holds fenced // materialization checkpoints, where supported by the endpoint and connector. -func FlowCheckpointsTable(path ...string) TableShape { - return TableShape{ - Path: path, +func FlowCheckpointsTable(base TablePath) *TableShape { + return &TableShape{ + Path: append(base, DefaultFlowCheckpoints), Binding: -1, Source: "", Comment: "This table holds Flow processing checkpoints used for exactly-once processing of materializations", @@ -93,58 +84,3 @@ func FlowCheckpointsTable(path ...string) TableShape { Document: nil, } } - -// FlowMaterializationsTable returns the TableShape that holds MaterializationSpecs. -func FlowMaterializationsTable(path ...string) TableShape { - - return TableShape{ - Path: path, - Binding: -1, - Source: "", - Comment: "This table is the source of truth for all materializations into this system.", - DeltaUpdates: false, - Keys: []Projection{ - { - Projection: flow.Projection{ - Field: "materialization", - Inference: flow.Inference{ - Types: []string{"string"}, - Exists: flow.Inference_MUST, - String_: &flow.Inference_String{}, - }, - }, - RawFieldConfig: nil, - Comment: "The name of the materialization.", - }, - }, - Values: []Projection{ - { - Projection: flow.Projection{ - Field: "version", - Inference: flow.Inference{ - Types: []string{"string"}, - Exists: flow.Inference_MUST, - String_: &flow.Inference_String{}, - }, - }, - RawFieldConfig: nil, - Comment: "Version of the materialization.", - }, - { - Projection: flow.Projection{ - Field: "spec", - Inference: flow.Inference{ - Types: []string{"string"}, - Exists: flow.Inference_MUST, - String_: &flow.Inference_String{ - ContentType: "application/x-protobuf; proto=flow.MaterializationSpec", - }, - }, - }, - RawFieldConfig: nil, - Comment: "Specification of the materialization, encoded as base64 protobuf.", - }, - }, - Document: nil, - } -} diff --git a/materialize-sql/std_sql.go b/materialize-sql/std_sql.go index 480948227f..1ea9f25220 100644 --- a/materialize-sql/std_sql.go +++ b/materialize-sql/std_sql.go @@ -10,33 +10,12 @@ import ( "slices" "strconv" "strings" + "time" boilerplate "github.com/estuary/connectors/materialize-boilerplate" - pf "github.com/estuary/flow/go/protocols/flow" log "github.com/sirupsen/logrus" ) -// StdFetchSpecAndVersion is a convenience for Client implementations which -// use Go's standard `sql.DB` type under the hood. -func StdFetchSpecAndVersion(ctx context.Context, db *sql.DB, specs Table, materialization pf.Materialization) (spec, version string, err error) { - // Fail-fast: surface a connection issue. - if err = db.PingContext(ctx); err != nil { - err = fmt.Errorf("connecting to DB: %w", err) - return - } - err = db.QueryRowContext( - ctx, - fmt.Sprintf( - "SELECT version, spec FROM %s WHERE materialization = %s;", - specs.Identifier, - specs.Keys[0].Placeholder, - ), - materialization.String(), - ).Scan(&version, &spec) - - return -} - // StdSQLExecStatements is a convenience for Client implementations which // use Go's standard `sql.DB` type under the hood. func StdSQLExecStatements(ctx context.Context, db *sql.DB, statements []string) error { @@ -255,6 +234,10 @@ func (col *anyColumn) Scan(i interface{}) error { switch ii := i.(type) { case []byte: sval = string(ii) + case time.Time: + // Consistent formatting of datetimes, which may otherwise use the + // database or local timezone. + sval = ii.UTC().Format(time.RFC3339Nano) case string: if _, err := strconv.Atoi(ii); err == nil { // Snowflake integer value columns scan into an interface{} with a concrete type of diff --git a/materialize-sql/templating_test.go b/materialize-sql/templating_test.go index c37afc38f9..f9ca88d43f 100644 --- a/materialize-sql/templating_test.go +++ b/materialize-sql/templating_test.go @@ -54,9 +54,9 @@ func newTestDialect() Dialect { func TestTableTemplate(t *testing.T) { var ( - shape = FlowCheckpointsTable("one", "reserved", "checkpoints") + shape = FlowCheckpointsTable([]string{"one", "reserved"}) dialect = newTestDialect() - table, err = ResolveTable(shape, dialect) + table, err = ResolveTable(*shape, dialect) ) assert.NoError(t, err) diff --git a/materialize-sql/test_support.go b/materialize-sql/test_support.go index fdc552f156..3941a2924a 100644 --- a/materialize-sql/test_support.go +++ b/materialize-sql/test_support.go @@ -42,8 +42,9 @@ func RunFenceTestCases( var runTest = func(t *testing.T, ranges ...uint32) { var ctx = context.Background() - var metaShape = FlowCheckpointsTable(checkpointsPath...) - var metaTable, err = ResolveTable(metaShape, dialect) + var metaShape = FlowCheckpointsTable(checkpointsPath) + metaShape.Path = checkpointsPath + var metaTable, err = ResolveTable(*metaShape, dialect) require.NoError(t, err) createSQL, err := RenderTableTemplate(metaTable, createTableTpl) @@ -319,7 +320,7 @@ func RunValidateAndApplyMigrationsTests( dumpSchema func(t *testing.T) string, insertData func(t *testing.T, cols []string, values []string), dumpData func(t *testing.T) string, - cleanup func(t *testing.T, materialization pf.Materialization), + cleanup func(t *testing.T), ) { ctx := context.Background() var snap strings.Builder @@ -331,7 +332,7 @@ func RunValidateAndApplyMigrationsTests( require.NoError(t, err) t.Run("validate and apply migratable type changes", func(t *testing.T) { - defer cleanup(t, pf.Materialization("test/sqlite")) + defer cleanup(t) fixture := loadValidateSpec(t, "base.flow.proto") @@ -404,7 +405,6 @@ func DumpTestTable(t *testing.T, db *stdsql.DB, qualifiedTableName string) (stri b.WriteString(col) b.WriteString(" (" + colTypes[i].DatabaseTypeName() + ")") } - b.WriteString("\n") for rows.Next() { var data = make([]anyColumn, len(cols)) diff --git a/materialize-sqlite/sqlite.go b/materialize-sqlite/sqlite.go index a0126c83da..ab230b64a9 100644 --- a/materialize-sqlite/sqlite.go +++ b/materialize-sqlite/sqlite.go @@ -79,7 +79,6 @@ func NewSQLiteDriver() *sql.Driver { return &sql.Endpoint{ Config: config{path: path}, Dialect: sqliteDialect, - MetaSpecs: nil, MetaCheckpoints: nil, NewClient: newClient, CreateTableTemplate: tplCreateTargetTable, @@ -125,16 +124,6 @@ func (c *client) DeleteTable(ctx context.Context, path []string) (string, boiler return "", nil, nil } -// We don't use specs table for sqlite since it is ephemeral and won't be -// persisted between ApplyUpsert and Transactions calls -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (specB64, version string, err error) { - return "", "", stdsql.ErrNoRows -} - -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - return nil -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-sqlserver/.snapshots/TestValidateAndApply b/materialize-sqlserver/.snapshots/TestValidateAndApply index 88d758bb0d..7793d6ef72 100644 --- a/materialize-sqlserver/.snapshots/TestValidateAndApply +++ b/materialize-sqlserver/.snapshots/TestValidateAndApply @@ -82,21 +82,21 @@ Big Schema Changed Types Constraints: {"Field":"boolField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolField' is already being materialized as endpoint type 'BIT' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} {"Field":"flow_published_at","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"intField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'intField' is already being materialized as endpoint type 'BIGINT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} +{"Field":"intField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"key","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is a key in the current materialization"} {"Field":"multipleField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"nullField","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Object fields may be materialized"} {"Field":"numField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'numField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'BIT' is required by its schema '{ type: [boolean] }'"} {"Field":"objField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringDateField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateField' is already being materialized as endpoint type 'DATE' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} -{"Field":"stringDateTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringDateTimeField' is already being materialized as endpoint type 'DATETIME2' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} +{"Field":"stringDateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringDateTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringDurationField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringField' is already being materialized as endpoint type 'VARCHAR' but endpoint type 'BIGINT' is required by its schema '{ type: [integer] }'"} {"Field":"stringHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnEmailField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIdnHostnameField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringIntegerField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringIntegerField' is already being materialized as endpoint type 'BIGINT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} +{"Field":"stringIntegerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv4Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIpv6Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringIriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} @@ -104,12 +104,12 @@ Big Schema Changed Types Constraints: {"Field":"stringJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddr8Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringMacAddrField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringNumberField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringNumberField' is already being materialized as endpoint type 'FLOAT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} +{"Field":"stringNumberField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRegexField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringRelativeJsonPointerField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} -{"Field":"stringTimeField","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringTimeField' is already being materialized as endpoint type 'TIME' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint32Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint32Field' is already being materialized as endpoint type 'BIGINT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} -{"Field":"stringUint64Field","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'stringUint64Field' is already being materialized as endpoint type 'BIGINT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [string] }'"} +{"Field":"stringTimeField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint32Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"stringUint64Field","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriReferenceField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"stringUriTemplateField","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} diff --git a/materialize-sqlserver/.snapshots/TestValidateAndApplyMigrations b/materialize-sqlserver/.snapshots/TestValidateAndApplyMigrations index 2c87a498f7..5972a2f934 100644 --- a/materialize-sqlserver/.snapshots/TestValidateAndApplyMigrations +++ b/materialize-sqlserver/.snapshots/TestValidateAndApplyMigrations @@ -1,5 +1,6 @@ Base Initial Constraints: {"Field":"_meta/flow_truncated","Type":4,"TypeString":"FIELD_OPTIONAL","Reason":"Metadata fields are able to be materialized"} +{"Field":"boolWidenedToJson","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"The projection has a single scalar type"} {"Field":"dateValue","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"The projection has a single scalar type"} {"Field":"datetimeValue","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"The projection has a single scalar type"} {"Field":"flow_document","Type":2,"TypeString":"LOCATION_REQUIRED","Reason":"The root document must be materialized"} @@ -20,6 +21,7 @@ Base Initial Constraints: Migratable Changes Before Apply Schema: {"Name":"_meta/flow_truncated","Nullable":"NO","Type":"bit"} +{"Name":"boolWidenedToJson","Nullable":"YES","Type":"bit"} {"Name":"dateValue","Nullable":"YES","Type":"date"} {"Name":"datetimeValue","Nullable":"YES","Type":"datetime2"} {"Name":"flow_document","Nullable":"NO","Type":"varchar"} @@ -38,12 +40,12 @@ Migratable Changes Before Apply Schema: Migratable Changes Before Apply Data: -key (NVARCHAR), _meta/flow_truncated (BIT), dateValue (DATE), datetimeValue (DATETIME2), flow_published_at (DATETIME2), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (NVARCHAR), nonScalarValue (NVARCHAR), numericString (BIGINT), optional (NVARCHAR), requiredNumeric (BIGINT), scalarValue (NVARCHAR), stringWidenedToJson (NVARCHAR), timeValue (TIME), flow_document (NVARCHAR) - -1, false, 2024-01-01 00:00:00 +0000 UTC, 2024-01-01 01:01:01.1111111 +0000 UTC, 2024-09-13 01:01:01 +0000 UTC, 1, 999, , , 123, , 456, test, hello, 0001-01-01 01:01:01 +0000 UTC, {} +key (NVARCHAR), _meta/flow_truncated (BIT), boolWidenedToJson (BIT), dateValue (DATE), datetimeValue (DATETIME2), flow_published_at (DATETIME2), int64 (BIGINT), intWidenedToJson (BIGINT), multiple (NVARCHAR), nonScalarValue (NVARCHAR), numericString (BIGINT), optional (NVARCHAR), requiredNumeric (BIGINT), scalarValue (NVARCHAR), stringWidenedToJson (NVARCHAR), timeValue (TIME), flow_document (NVARCHAR) +1, false, true, 2024-01-01T00:00:00Z, 2024-01-01T01:01:01.1111111Z, 2024-09-13T01:01:01Z, 1, 999, , , 123, , 456, test, hello, 0001-01-01T01:01:01Z, {} Migratable Changes Constraints: {"Field":"_meta/flow_truncated","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} +{"Field":"boolWidenedToJson","Type":6,"TypeString":"UNSATISFIABLE","Reason":"Field 'boolWidenedToJson' is already being materialized as endpoint type 'BIT' but endpoint type 'VARCHAR(MAX) COLLATE LATIN1_GENERAL_100_BIN2_UTF8' is required by its schema '{ type: [boolean, string] }'"} {"Field":"dateValue","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"datetimeValue","Type":3,"TypeString":"LOCATION_RECOMMENDED","Reason":"This location is part of the current materialization"} {"Field":"flow_document","Type":1,"TypeString":"FIELD_REQUIRED","Reason":"This field is the document in the current materialization"} @@ -64,6 +66,7 @@ Migratable Changes Constraints: Migratable Changes Applied Schema: {"Name":"_meta/flow_truncated","Nullable":"NO","Type":"bit"} +{"Name":"boolWidenedToJson","Nullable":"YES","Type":"bit"} {"Name":"dateValue","Nullable":"YES","Type":"varchar"} {"Name":"datetimeValue","Nullable":"YES","Type":"varchar"} {"Name":"flow_document","Nullable":"NO","Type":"varchar"} @@ -82,7 +85,6 @@ Migratable Changes Applied Schema: Migratable Changes Applied Data: -key (NVARCHAR), _meta/flow_truncated (BIT), flow_published_at (DATETIME2), int64 (BIGINT), multiple (NVARCHAR), nonScalarValue (NVARCHAR), optional (NVARCHAR), scalarValue (NVARCHAR), stringWidenedToJson (NVARCHAR), flow_document (NVARCHAR), dateValue (NVARCHAR), datetimeValue (NVARCHAR), intWidenedToJson (NVARCHAR), numericString (NVARCHAR), requiredNumeric (NVARCHAR), timeValue (NVARCHAR) - -1, false, 2024-09-13 01:01:01 +0000 UTC, 1, , , , test, hello, {}, 2024-01-01, 2024-01-01T01:01:01.1111111Z, 999, 123, 456, 01:01:01.0000000 +key (NVARCHAR), _meta/flow_truncated (BIT), boolWidenedToJson (BIT), flow_published_at (DATETIME2), int64 (BIGINT), multiple (NVARCHAR), nonScalarValue (NVARCHAR), optional (NVARCHAR), scalarValue (NVARCHAR), stringWidenedToJson (NVARCHAR), flow_document (NVARCHAR), dateValue (NVARCHAR), datetimeValue (NVARCHAR), intWidenedToJson (NVARCHAR), numericString (NVARCHAR), requiredNumeric (NVARCHAR), timeValue (NVARCHAR) +1, false, true, 2024-09-13T01:01:01Z, 1, , , , test, hello, {}, 2024-01-01, 2024-01-01T01:01:01.1111111Z, 999, 123, 456, 01:01:01.0000000 diff --git a/materialize-sqlserver/client.go b/materialize-sqlserver/client.go index d9cd489d2e..db1a9da90a 100644 --- a/materialize-sqlserver/client.go +++ b/materialize-sqlserver/client.go @@ -14,7 +14,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - "github.com/estuary/flow/go/protocols/flow" mssqldb "github.com/microsoft/go-mssqldb" ) @@ -222,11 +221,6 @@ func (c *client) DeleteTable(ctx context.Context, path []string) (string, boiler }, nil } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - _, err := c.db.ExecContext(ctx, updateSpec.ParameterizedQuery, updateSpec.Parameters...) - return err -} - func (c *client) InstallFence(ctx context.Context, checkpoints sql.Table, fence sql.Fence) (sql.Fence, error) { return installFence(ctx, c.ep.Dialect, c.db, checkpoints, fence, base64.StdEncoding.DecodeString) } @@ -235,10 +229,6 @@ func (c *client) ExecStatements(ctx context.Context, statements []string) error return sql.StdSQLExecStatements(ctx, c.db, statements) } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization flow.Materialization) (string, string, error) { - return sql.StdFetchSpecAndVersion(ctx, c.db, specs, materialization) -} - func (c *client) Close() { c.db.Close() } diff --git a/materialize-sqlserver/driver.go b/materialize-sqlserver/driver.go index 40a7758e04..457eada0f7 100644 --- a/materialize-sqlserver/driver.go +++ b/materialize-sqlserver/driver.go @@ -181,8 +181,6 @@ func newSqlServerDriver() *sql.Driver { if cfg.Schema != "" { metaBase = append(metaBase, cfg.Schema) } - var metaSpecs, metaCheckpoints = sql.MetaTables(metaBase) - db, err := stdsql.Open("sqlserver", cfg.ToURI()) if err != nil { return nil, fmt.Errorf("opening db: %w", err) @@ -209,8 +207,7 @@ func newSqlServerDriver() *sql.Driver { return &sql.Endpoint{ Config: cfg, Dialect: dialect, - MetaSpecs: &metaSpecs, - MetaCheckpoints: &metaCheckpoints, + MetaCheckpoints: sql.FlowCheckpointsTable(metaBase), NewClient: newClient, CreateTableTemplate: templates.createTargetTable, NewResource: newTableConfig, diff --git a/materialize-sqlserver/driver_test.go b/materialize-sqlserver/driver_test.go index df81afdead..22cd53e286 100644 --- a/materialize-sqlserver/driver_test.go +++ b/materialize-sqlserver/driver_test.go @@ -59,15 +59,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = 'test/sqlite'", - testDialect.Identifier("flow_materializations_v2"), - )) }, ) } @@ -105,6 +99,14 @@ func TestValidateAndApplyMigrations(t *testing.T) { for i, col := range cols { keys[i] = testDialect.Identifier(col) } + for i := range values { + if values[i] == "true" { + values[i] = "1" + } else if values[i] == "false" { + values[i] = "0" + } + } + keys = append(keys, testDialect.Identifier("_meta/flow_truncated")) values = append(values, "0") keys = append(keys, testDialect.Identifier("flow_published_at")) @@ -125,16 +127,9 @@ func TestValidateAndApplyMigrations(t *testing.T) { return rows }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s;", testDialect.Identifier(resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - testDialect.Identifier(sql.DefaultFlowMaterializations), - testDialect.Literal(materialization.String()), - )) }, ) } @@ -291,14 +286,6 @@ func TestPrereqs(t *testing.T) { }, want: []string{"Cannot open database \"wrongmaster\" that was requested by the login."}, }, - { - name: "wrong address", - cfg: func(cfg config) *config { - cfg.Address = "wrong." + cfg.Address - return &cfg - }, - want: []string{fmt.Sprintf("host at address %q cannot be found", "wrong."+cfg.Address)}, - }, } ctx := context.Background() diff --git a/materialize-starburst/client.go b/materialize-starburst/client.go index 924340407e..9818115664 100644 --- a/materialize-starburst/client.go +++ b/materialize-starburst/client.go @@ -9,7 +9,6 @@ import ( boilerplate "github.com/estuary/connectors/materialize-boilerplate" sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" ) var _ sql.SchemaManager = (*client)(nil) @@ -111,13 +110,6 @@ func (c *client) InfoSchema(ctx context.Context, resourcePaths [][]string) (*boi return is, nil } -func (c *client) PutSpec(ctx context.Context, updateSpec sql.MetaSpecsUpdate) error { - // Normalize query by removing trailing ';' as Trino does not accept it. - updateSpec.QueryString = strings.TrimRight(updateSpec.QueryString, ";") - _, err := c.db.ExecContext(ctx, updateSpec.QueryString) - return err -} - func (c *client) CreateTable(ctx context.Context, tc sql.TableCreate) error { _, err := c.db.ExecContext(ctx, tc.TableCreateSql) return err @@ -186,34 +178,6 @@ func preReqs(ctx context.Context, conf any, tenant string) *sql.PrereqErr { return errs } -func (c *client) FetchSpecAndVersion(ctx context.Context, specs sql.Table, materialization pf.Materialization) (spec, version string, err error) { - - fetchVersionAndSpecQuery, err := sql.RenderTableTemplate(specs, c.templates.fetchVersionAndSpec) - if err != nil { - return "", "", err - } - - // QueryRowContext cannot be used until Trino driver issue is fixed https://github.com/trinodb/trino-go-client/issues/102 - rows, err := c.db.QueryContext(ctx, - fetchVersionAndSpecQuery, - materialization.String()) - if err != nil { - return "", "", fmt.Errorf("quering spec and version faield: %w", err) - } - var numberOfResults int - for rows.Next() { - numberOfResults++ - if err := rows.Scan(&version, &spec); err != nil { - return "", "", fmt.Errorf("quering spec and version faield: %w", err) - } - } - if numberOfResults != 1 { - return "", "", fmt.Errorf("quering spec and version should return exactly one result number of results: %d", numberOfResults) - } - - return -} - func (c *client) ExecStatements(ctx context.Context, statements []string) error { return sql.StdSQLExecStatements(ctx, c.db, statements) } diff --git a/materialize-starburst/starburst.go b/materialize-starburst/starburst.go index e63930df82..9806b63651 100644 --- a/materialize-starburst/starburst.go +++ b/materialize-starburst/starburst.go @@ -117,14 +117,11 @@ func newStarburstDriver() *sql.Driver { "schema": cfg.Schema, }).Info("opening Starburst") - var metaBase sql.TablePath - var metaSpecs, _ = sql.MetaTables(metaBase) var templates = renderTemplates(starburstTrinoDialect) return &sql.Endpoint{ Config: cfg, Dialect: starburstTrinoDialect, - MetaSpecs: &metaSpecs, NewClient: newClient, CreateTableTemplate: templates.createTargetTable, NewResource: newTableConfig, diff --git a/materialize-starburst/starburst_test.go b/materialize-starburst/starburst_test.go index 0406d18973..d250310d63 100644 --- a/materialize-starburst/starburst_test.go +++ b/materialize-starburst/starburst_test.go @@ -7,16 +7,15 @@ import ( stdsql "database/sql" "encoding/json" "fmt" - "github.com/bradleyjkemp/cupaloy" - boilerplate "github.com/estuary/connectors/materialize-boilerplate" - sql "github.com/estuary/connectors/materialize-sql" - pf "github.com/estuary/flow/go/protocols/flow" - pm "github.com/estuary/flow/go/protocols/materialize" "os" "slices" "strings" "testing" + "github.com/bradleyjkemp/cupaloy" + boilerplate "github.com/estuary/connectors/materialize-boilerplate" + pm "github.com/estuary/flow/go/protocols/materialize" + "github.com/stretchr/testify/require" ) @@ -80,16 +79,9 @@ func TestValidateAndApply(t *testing.T) { return sch }, - func(t *testing.T, materialization pf.Materialization) { + func(t *testing.T) { t.Helper() - _, _ = db.ExecContext(ctx, fmt.Sprintf("drop table %s", targetTableDialect.Identifier(resourceConfig.Schema, resourceConfig.Table))) - - _, _ = db.ExecContext(ctx, fmt.Sprintf( - "delete from %s where materialization = %s", - targetTableDialect.Identifier(cfg.Schema, sql.DefaultFlowMaterializations), - targetTableDialect.Literal(materialization.String()), - )) }, ) } diff --git a/tests/materialize/materialize-bigquery/snapshot.json b/tests/materialize/materialize-bigquery/snapshot.json index b7db5b7627..46d7543804 100644 --- a/tests/materialize/materialize-bigquery/snapshot.json +++ b/tests/materialize/materialize-bigquery/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "CREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.simple (\n\t\tid INTEGER NOT NULL,\n\t\tcanary STRING NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.duplicate_keys (\n\t\tid INTEGER NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tint INTEGER,\n\t\tstr STRING NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.multiple_types (\n\t\tid INTEGER NOT NULL,\n\t\tarray_int STRING,\n\t\tbinary_field STRING,\n\t\tbool_field BOOLEAN,\n\t\tfloat_field FLOAT64,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tmultiple JSON,\n\t\tnested STRING,\n\t\tnullable_int INTEGER,\n\t\tstr_field STRING NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.formatted_strings (\n\t\tid INTEGER NOT NULL,\n\t\tdate DATE,\n\t\tdatetime TIMESTAMP,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tint_and_str BIGNUMERIC(38,0),\n\t\tint_str BIGNUMERIC(38,0),\n\t\tnum_and_str FLOAT64,\n\t\tnum_str FLOAT64,\n\t\ttime STRING,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.unsigned_bigint (\n\t\tid INTEGER NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tunsigned_bigint BIGNUMERIC(38,0),\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.deletions (\n\t\tid INTEGER NOT NULL,\n\t\t_meta_op STRING,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nINSERT INTO `estuary-theatre`.testing.flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-bigquery/materialize');" + "CREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.simple (\n\t\tid INTEGER NOT NULL,\n\t\tcanary STRING NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.duplicate_keys (\n\t\tid INTEGER NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tint INTEGER,\n\t\tstr STRING NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.multiple_types (\n\t\tid INTEGER NOT NULL,\n\t\tarray_int STRING,\n\t\tbinary_field STRING,\n\t\tbool_field BOOLEAN,\n\t\tfloat_field FLOAT64,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tmultiple JSON,\n\t\tnested STRING,\n\t\tnullable_int INTEGER,\n\t\tstr_field STRING NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.formatted_strings (\n\t\tid INTEGER NOT NULL,\n\t\tdate DATE,\n\t\tdatetime TIMESTAMP,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tint_and_str BIGNUMERIC(38,0),\n\t\tint_str BIGNUMERIC(38,0),\n\t\tnum_and_str FLOAT64,\n\t\tnum_str FLOAT64,\n\t\ttime STRING,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.unsigned_bigint (\n\t\tid INTEGER NOT NULL,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tunsigned_bigint BIGNUMERIC(38,0),\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n\nCREATE TABLE IF NOT EXISTS `estuary-theatre`.testing.deletions (\n\t\tid INTEGER NOT NULL,\n\t\t_meta_op STRING,\n\t\tflow_published_at TIMESTAMP NOT NULL,\n\t\tflow_document STRING NOT NULL\n)\nCLUSTER BY id;\n" ] [ "connectorState", @@ -351,7 +351,7 @@ } [ "applied.actionDescription", - "UPDATE `estuary-theatre`.testing.flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-bigquery/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-databricks/snapshot.json b/tests/materialize/materialize-databricks/snapshot.json index 0056ff7f7a..0653f24bec 100644 --- a/tests/materialize/materialize-databricks/snapshot.json +++ b/tests/materialize/materialize-databricks/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS `some-schema`.simple (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n canary STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /canary with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/simple' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_standard (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_delta (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_delta_exclude_flow_doc (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.multiple_types (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n array_int STRING COMMENT 'auto-generated projection of JSON at: /array_int with inferred types: [array]',\n binary_field BINARY COMMENT 'auto-generated projection of JSON at: /binary_field with inferred types: [string]',\n bool_field BOOLEAN COMMENT 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]',\n float_field DOUBLE COMMENT 'auto-generated projection of JSON at: /float_field with inferred types: [number]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n multiple STRING COMMENT 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]',\n nested STRING COMMENT 'auto-generated projection of JSON at: /nested with inferred types: [object]',\n nullable_int LONG COMMENT 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]',\n str_field STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/multiple-data-types' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.formatted_strings (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n date DATE COMMENT 'auto-generated projection of JSON at: /date with inferred types: [string]',\n datetime TIMESTAMP COMMENT 'auto-generated projection of JSON at: /datetime with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int_and_str NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]',\n int_str NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /int_str with inferred types: [string]',\n num_and_str DOUBLE COMMENT 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]',\n num_str DOUBLE COMMENT 'auto-generated projection of JSON at: /num_str with inferred types: [string]',\n time STRING COMMENT 'auto-generated projection of JSON at: /time with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/formatted-strings' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.unsigned_bigint (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n unsigned_bigint NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/unsigned-bigint' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.deletions (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n `_meta/op` STRING COMMENT 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/deletions' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.binary_key (\n id BINARY NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [string]',\n counter LONG COMMENT 'auto-generated projection of JSON at: /counter with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/binary-key' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\nINSERT INTO `some-schema`.flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-databricks/materialize');" + "\nCREATE TABLE IF NOT EXISTS `some-schema`.simple (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n canary STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /canary with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/simple' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_standard (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_delta (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.duplicate_keys_delta_exclude_flow_doc (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int LONG COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n str STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/duplicated-keys' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.multiple_types (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n array_int STRING COMMENT 'auto-generated projection of JSON at: /array_int with inferred types: [array]',\n binary_field BINARY COMMENT 'auto-generated projection of JSON at: /binary_field with inferred types: [string]',\n bool_field BOOLEAN COMMENT 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]',\n float_field DOUBLE COMMENT 'auto-generated projection of JSON at: /float_field with inferred types: [number]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n multiple STRING COMMENT 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]',\n nested STRING COMMENT 'auto-generated projection of JSON at: /nested with inferred types: [object]',\n nullable_int LONG COMMENT 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]',\n str_field STRING NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/multiple-data-types' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.formatted_strings (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n date DATE COMMENT 'auto-generated projection of JSON at: /date with inferred types: [string]',\n datetime TIMESTAMP COMMENT 'auto-generated projection of JSON at: /datetime with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n int_and_str NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]',\n int_str NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /int_str with inferred types: [string]',\n num_and_str DOUBLE COMMENT 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]',\n num_str DOUBLE COMMENT 'auto-generated projection of JSON at: /num_str with inferred types: [string]',\n time STRING COMMENT 'auto-generated projection of JSON at: /time with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/formatted-strings' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.unsigned_bigint (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n unsigned_bigint NUMERIC(38,0) COMMENT 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/unsigned-bigint' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.deletions (\n id LONG NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n `_meta/op` STRING COMMENT 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/deletions' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n\n\nCREATE TABLE IF NOT EXISTS `some-schema`.binary_key (\n id BINARY NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [string]',\n counter LONG COMMENT 'auto-generated projection of JSON at: /counter with inferred types: [integer]',\n flow_published_at TIMESTAMP NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n flow_document STRING NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) COMMENT 'Generated for materialization tests/materialize-databricks/materialize of collection tests/binary-key' TBLPROPERTIES ('delta.columnMapping.mode' = 'name');\n" ] [ "connectorState", @@ -698,7 +698,7 @@ } [ "applied.actionDescription", - "UPDATE `some-schema`.flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-databricks/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-elasticsearch/snapshot.json b/tests/materialize/materialize-elasticsearch/snapshot.json index c4c580b5f0..12f2b90504 100644 --- a/tests/materialize/materialize-elasticsearch/snapshot.json +++ b/tests/materialize/materialize-elasticsearch/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "create index \"flow_materializations_v2\"\ncreate index \"index-simple\"\ncreate index \"index-duplicated-keys-standard\"\ncreate index \"index-duplicated-keys-delta\"\ncreate index \"index-duplicated-keys-delta-exclude-flow-doc\"\ncreate index \"index-multiple-data-types\"\ncreate index \"index-formatted-strings\"\ncreate index \"index-deletions\"\nupdate stored materialization spec and set version = ffffffffffffffff" + "create index \"index-simple\"\ncreate index \"index-duplicated-keys-standard\"\ncreate index \"index-duplicated-keys-delta\"\ncreate index \"index-duplicated-keys-delta-exclude-flow-doc\"\ncreate index \"index-multiple-data-types\"\ncreate index \"index-formatted-strings\"\ncreate index \"index-deletions\"" ] [ "connectorState", @@ -947,7 +947,7 @@ } [ "applied.actionDescription", - "create index \"flow_materializations_v2\"\nupdate stored materialization spec and set version = ffffffffffffffff" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-mongodb/snapshot.json b/tests/materialize/materialize-mongodb/snapshot.json index 32f4198d5f..a3b00ccd94 100644 --- a/tests/materialize/materialize-mongodb/snapshot.json +++ b/tests/materialize/materialize-mongodb/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "update persisted spec" + "" ] [ "connectorState", @@ -260,7 +260,7 @@ } [ "applied.actionDescription", - "update persisted spec" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-motherduck/snapshot.json b/tests/materialize/materialize-motherduck/snapshot.json index 6874474d08..c70467adac 100644 --- a/tests/materialize/materialize-motherduck/snapshot.json +++ b/tests/materialize/materialize-motherduck/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS test_db.main.\"simple\" (\n\tid BIGINT NOT NULL,\n\tcanary VARCHAR NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_standard (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_delta (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_delta_exclude_flow_doc (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.multiple_types (\n\tid BIGINT NOT NULL,\n\tarray_int JSON,\n\tbinary_field VARCHAR,\n\tbool_field BOOLEAN,\n\tfloat_field DOUBLE,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tmultiple JSON,\n\tnested JSON,\n\tnullable_int BIGINT,\n\tstr_field VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.formatted_strings (\n\tid BIGINT NOT NULL,\n\tdate DATE,\n\tdatetime TIMESTAMP WITH TIME ZONE,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tint_and_str HUGEINT,\n\tint_str HUGEINT,\n\tnum_and_str DOUBLE,\n\tnum_str DOUBLE,\n\t\"time\" TIME,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.unsigned_bigint (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tunsigned_bigint HUGEINT,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.deletions (\n\tid BIGINT NOT NULL,\n\t\"_meta/op\" VARCHAR,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\nINSERT INTO test_db.main.flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-motherduck/materialize');" + "\nCREATE TABLE IF NOT EXISTS test_db.main.\"simple\" (\n\tid BIGINT NOT NULL,\n\tcanary VARCHAR NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_standard (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_delta (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.duplicate_keys_delta_exclude_flow_doc (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\t\"int\" BIGINT,\n\tstr VARCHAR NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.multiple_types (\n\tid BIGINT NOT NULL,\n\tarray_int JSON,\n\tbinary_field VARCHAR,\n\tbool_field BOOLEAN,\n\tfloat_field DOUBLE,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tmultiple JSON,\n\tnested JSON,\n\tnullable_int BIGINT,\n\tstr_field VARCHAR NOT NULL,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.formatted_strings (\n\tid BIGINT NOT NULL,\n\tdate DATE,\n\tdatetime TIMESTAMP WITH TIME ZONE,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tint_and_str HUGEINT,\n\tint_str HUGEINT,\n\tnum_and_str DOUBLE,\n\tnum_str DOUBLE,\n\t\"time\" TIME,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.unsigned_bigint (\n\tid BIGINT NOT NULL,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tunsigned_bigint HUGEINT,\n\tflow_document JSON NOT NULL\n);\n\n\nCREATE TABLE IF NOT EXISTS test_db.main.deletions (\n\tid BIGINT NOT NULL,\n\t\"_meta/op\" VARCHAR,\n\tflow_published_at TIMESTAMP WITH TIME ZONE NOT NULL,\n\tflow_document JSON NOT NULL\n);\n" ] [ "connectorState", @@ -1148,7 +1148,7 @@ } [ "applied.actionDescription", - "UPDATE test_db.main.flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-motherduck/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-mysql/snapshot.json b/tests/materialize/materialize-mysql/snapshot.json index 2a8dcb9bf1..2d2992e6bc 100644 --- a/tests/materialize/materialize-mysql/snapshot.json +++ b/tests/materialize/materialize-mysql/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS flow_materializations_v2 (\n\t\tmaterialization VARCHAR(256) NOT NULL COMMENT 'The name of the materialization.',\n\t\tversion LONGTEXT NOT NULL COMMENT 'Version of the materialization.',\n\t\tspec LONGBLOB NOT NULL COMMENT 'Specification of the materialization, encoded as base64 protobuf.',\n\n\t\tPRIMARY KEY (materialization)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='This table is the source of truth for all materializations into this system.';\n\n\nCREATE TABLE IF NOT EXISTS flow_checkpoints_v1 (\n\t\tmaterialization VARCHAR(256) NOT NULL COMMENT 'The name of the materialization.',\n\t\tkey_begin BIGINT NOT NULL COMMENT 'The inclusive lower-bound key hash covered by this checkpoint.',\n\t\tkey_end BIGINT NOT NULL COMMENT 'The inclusive upper-bound key hash covered by this checkpoint.',\n\t\tfence BIGINT NOT NULL COMMENT 'This nonce is used to uniquely identify unique process assignments of a shard and prevent them from conflicting.',\n\t\tcheckpoint LONGBLOB NOT NULL COMMENT 'Checkpoint of the Flow consumer shard, encoded as base64 protobuf.',\n\n\t\tPRIMARY KEY (materialization, key_begin, key_end)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='This table holds Flow processing checkpoints used for exactly-once processing of materializations';\n\n\nCREATE TABLE IF NOT EXISTS `Simple` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tcanary LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /canary with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/simple';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]'\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS `Multiple Types` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tarray_int JSON COMMENT 'auto-generated projection of JSON at: /array_int with inferred types: [array]',\n\t\tbinary_field LONGTEXT COMMENT 'auto-generated projection of JSON at: /binary_field with inferred types: [string]',\n\t\tbool_field BOOLEAN COMMENT 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]',\n\t\tfloat_field DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /float_field with inferred types: [number]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tmultiple JSON COMMENT 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]',\n\t\t`nested` JSON COMMENT 'auto-generated projection of JSON at: /nested with inferred types: [object]',\n\t\tnullable_int BIGINT COMMENT 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]',\n\t\tstr_field LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/multiple-data-types';\n\n\nCREATE TABLE IF NOT EXISTS `Formatted Strings` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\t`date` DATE COMMENT 'auto-generated projection of JSON at: /date with inferred types: [string]',\n\t\t`datetime` DATETIME(6) COMMENT 'auto-generated projection of JSON at: /datetime with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tint_and_str NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]',\n\t\tint_str NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /int_str with inferred types: [string]',\n\t\tnum_and_str DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]',\n\t\tnum_str DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /num_str with inferred types: [string]',\n\t\t`time` TIME(6) COMMENT 'auto-generated projection of JSON at: /time with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/formatted-strings';\n\n\nCREATE TABLE IF NOT EXISTS `long-string` (\n\t\tid VARCHAR(256) NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tstr_field LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/long-string';\n\n\nCREATE TABLE IF NOT EXISTS unsigned_bigint (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tunsigned_bigint NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/unsigned-bigint';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\t`_meta/op` LONGTEXT COMMENT 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/deletions';\n\nINSERT INTO flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-mysql/materialize');" + "\nCREATE TABLE IF NOT EXISTS `Simple` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tcanary LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /canary with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/simple';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]'\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\t`int` BIGINT COMMENT 'auto-generated projection of JSON at: /int with inferred types: [integer]',\n\t\tstr LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str with inferred types: [string]'\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/duplicated-keys';\n\n\nCREATE TABLE IF NOT EXISTS `Multiple Types` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tarray_int JSON COMMENT 'auto-generated projection of JSON at: /array_int with inferred types: [array]',\n\t\tbinary_field LONGTEXT COMMENT 'auto-generated projection of JSON at: /binary_field with inferred types: [string]',\n\t\tbool_field BOOLEAN COMMENT 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]',\n\t\tfloat_field DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /float_field with inferred types: [number]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tmultiple JSON COMMENT 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]',\n\t\t`nested` JSON COMMENT 'auto-generated projection of JSON at: /nested with inferred types: [object]',\n\t\tnullable_int BIGINT COMMENT 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]',\n\t\tstr_field LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/multiple-data-types';\n\n\nCREATE TABLE IF NOT EXISTS `Formatted Strings` (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\t`date` DATE COMMENT 'auto-generated projection of JSON at: /date with inferred types: [string]',\n\t\t`datetime` DATETIME(6) COMMENT 'auto-generated projection of JSON at: /datetime with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tint_and_str NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]',\n\t\tint_str NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /int_str with inferred types: [string]',\n\t\tnum_and_str DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]',\n\t\tnum_str DOUBLE PRECISION COMMENT 'auto-generated projection of JSON at: /num_str with inferred types: [string]',\n\t\t`time` TIME(6) COMMENT 'auto-generated projection of JSON at: /time with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/formatted-strings';\n\n\nCREATE TABLE IF NOT EXISTS `long-string` (\n\t\tid VARCHAR(256) NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tstr_field LONGTEXT NOT NULL COMMENT 'auto-generated projection of JSON at: /str_field with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/long-string';\n\n\nCREATE TABLE IF NOT EXISTS unsigned_bigint (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tunsigned_bigint NUMERIC(65,0) COMMENT 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/unsigned-bigint';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\t\tid BIGINT NOT NULL COMMENT 'auto-generated projection of JSON at: /id with inferred types: [integer]',\n\t\t`_meta/op` LONGTEXT COMMENT 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]',\n\t\tflow_published_at DATETIME(6) NOT NULL COMMENT 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]',\n\t\tflow_document JSON NOT NULL COMMENT 'auto-generated projection of JSON at: with inferred types: [object]',\n\n\t\tPRIMARY KEY (id)\n) CHARACTER SET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='Generated for materialization tests/materialize-mysql/materialize of collection tests/deletions';\n" ] [ "connectorState", @@ -547,7 +547,7 @@ } [ "applied.actionDescription", - "UPDATE flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-mysql/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-postgres/snapshot.json b/tests/materialize/materialize-postgres/snapshot.json index d0f086eb93..cbb89496ef 100644 --- a/tests/materialize/materialize-postgres/snapshot.json +++ b/tests/materialize/materialize-postgres/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS \"public\".flow_materializations_v2 (\n\t\tmaterialization TEXT NOT NULL,\n\t\tversion TEXT NOT NULL,\n\t\tspec TEXT NOT NULL,\n\n\t\tPRIMARY KEY (materialization)\n);\n\nCOMMENT ON TABLE \"public\".flow_materializations_v2 IS 'This table is the source of truth for all materializations into this system.';\nCOMMENT ON COLUMN \"public\".flow_materializations_v2.materialization IS 'The name of the materialization.';\nCOMMENT ON COLUMN \"public\".flow_materializations_v2.version IS 'Version of the materialization.';\nCOMMENT ON COLUMN \"public\".flow_materializations_v2.spec IS 'Specification of the materialization, encoded as base64 protobuf.';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".flow_checkpoints_v1 (\n\t\tmaterialization TEXT NOT NULL,\n\t\tkey_begin BIGINT NOT NULL,\n\t\tkey_end BIGINT NOT NULL,\n\t\tfence BIGINT NOT NULL,\n\t\tcheckpoint TEXT NOT NULL,\n\n\t\tPRIMARY KEY (materialization, key_begin, key_end)\n);\n\nCOMMENT ON TABLE \"public\".flow_checkpoints_v1 IS 'This table holds Flow processing checkpoints used for exactly-once processing of materializations';\nCOMMENT ON COLUMN \"public\".flow_checkpoints_v1.materialization IS 'The name of the materialization.';\nCOMMENT ON COLUMN \"public\".flow_checkpoints_v1.key_begin IS 'The inclusive lower-bound key hash covered by this checkpoint.';\nCOMMENT ON COLUMN \"public\".flow_checkpoints_v1.key_end IS 'The inclusive upper-bound key hash covered by this checkpoint.';\nCOMMENT ON COLUMN \"public\".flow_checkpoints_v1.fence IS 'This nonce is used to uniquely identify unique process assignments of a shard and prevent them from conflicting.';\nCOMMENT ON COLUMN \"public\".flow_checkpoints_v1.checkpoint IS 'Checkpoint of the Flow consumer shard, encoded as base64 protobuf.';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".\"Simple\" (\n\t\tid BIGINT NOT NULL,\n\t\tcanary TEXT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Simple\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/simple';\nCOMMENT ON COLUMN \"public\".\"Simple\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Simple\".canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Simple\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Simple\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_standard (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_standard IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_delta (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_delta IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".\"Multiple Types\" (\n\t\tid BIGINT NOT NULL,\n\t\tarray_int JSON,\n\t\tbinary_field TEXT,\n\t\tbool_field BOOLEAN,\n\t\tfloat_field DOUBLE PRECISION,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tmultiple JSON,\n\t\tnested JSON,\n\t\tnullable_int BIGINT,\n\t\tstr_field TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Multiple Types\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".\"Formatted Strings\" (\n\t\tid BIGINT NOT NULL,\n\t\t\"date\" DATE,\n\t\tdatetime TIMESTAMPTZ,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tint_and_str NUMERIC,\n\t\tint_str NUMERIC,\n\t\tnum_and_str DECIMAL,\n\t\tnum_str DECIMAL,\n\t\t\"time\" TIME,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Formatted Strings\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".\"date\" IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".\"time\" IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".unsigned_bigint (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tunsigned_bigint NUMERIC,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".unsigned_bigint IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/unsigned-bigint';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.unsigned_bigint IS 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".deletions (\n\t\tid BIGINT NOT NULL,\n\t\t\"_meta/op\" TEXT,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".deletions IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/deletions';\nCOMMENT ON COLUMN \"public\".deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\nINSERT INTO \"public\".flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-postgres/materialize');" + "\nCREATE TABLE IF NOT EXISTS \"public\".\"Simple\" (\n\t\tid BIGINT NOT NULL,\n\t\tcanary TEXT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Simple\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/simple';\nCOMMENT ON COLUMN \"public\".\"Simple\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Simple\".canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Simple\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Simple\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_standard (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_standard IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_delta (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_delta IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr TEXT NOT NULL\n);\n\nCOMMENT ON TABLE \"public\".duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.\"int\" IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".\"Multiple Types\" (\n\t\tid BIGINT NOT NULL,\n\t\tarray_int JSON,\n\t\tbinary_field TEXT,\n\t\tbool_field BOOLEAN,\n\t\tfloat_field DOUBLE PRECISION,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tmultiple JSON,\n\t\tnested JSON,\n\t\tnullable_int BIGINT,\n\t\tstr_field TEXT NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Multiple Types\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Multiple Types\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".\"Formatted Strings\" (\n\t\tid BIGINT NOT NULL,\n\t\t\"date\" DATE,\n\t\tdatetime TIMESTAMPTZ,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tint_and_str NUMERIC,\n\t\tint_str NUMERIC,\n\t\tnum_and_str DECIMAL,\n\t\tnum_str DECIMAL,\n\t\t\"time\" TIME,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".\"Formatted Strings\" IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".\"date\" IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".\"time\" IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".\"Formatted Strings\".flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".unsigned_bigint (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tunsigned_bigint NUMERIC,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".unsigned_bigint IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/unsigned-bigint';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.unsigned_bigint IS 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".unsigned_bigint.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS \"public\".deletions (\n\t\tid BIGINT NOT NULL,\n\t\t\"_meta/op\" TEXT,\n\t\tflow_published_at TIMESTAMPTZ NOT NULL,\n\t\tflow_document JSON NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE \"public\".deletions IS 'Generated for materialization tests/materialize-postgres/materialize of collection tests/deletions';\nCOMMENT ON COLUMN \"public\".deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN \"public\".deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN \"public\".deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n" ] [ "connectorState", @@ -537,7 +537,7 @@ } [ "applied.actionDescription", - "UPDATE \"public\".flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-postgres/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-redshift/snapshot.json b/tests/materialize/materialize-redshift/snapshot.json index c37608d307..feb99f23ba 100644 --- a/tests/materialize/materialize-redshift/snapshot.json +++ b/tests/materialize/materialize-redshift/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS simple (\n\tid BIGINT,\n\tcanary TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE simple IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/simple';\nCOMMENT ON COLUMN simple.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN simple.canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE duplicate_keys_standard IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE duplicate_keys_delta IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT\n);\n\nCOMMENT ON TABLE duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS multiple_types (\n\tid BIGINT,\n\tarray_int SUPER,\n\tbinary_field TEXT,\n\tbool_field BOOLEAN,\n\tfloat_field DOUBLE PRECISION,\n\tflow_published_at TIMESTAMPTZ,\n\tmultiple SUPER,\n\tnested SUPER,\n\tnullable_int BIGINT,\n\tstr_field TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE multiple_types IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN multiple_types.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN multiple_types.array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN multiple_types.binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN multiple_types.float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN multiple_types.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN multiple_types.nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN multiple_types.nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN multiple_types.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS formatted_strings (\n\tid BIGINT,\n\tdate DATE,\n\tdatetime TIMESTAMPTZ,\n\tflow_published_at TIMESTAMPTZ,\n\tint_and_str NUMERIC(38,0),\n\tint_str NUMERIC(38,0),\n\tnum_and_str DOUBLE PRECISION,\n\tnum_str DOUBLE PRECISION,\n\ttime TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE formatted_strings IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN formatted_strings.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN formatted_strings.date IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN formatted_strings.int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN formatted_strings.num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.time IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS long_string (\n\tid TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tstr_field TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE long_string IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/long-string';\nCOMMENT ON COLUMN long_string.id IS 'auto-generated projection of JSON at: /id with inferred types: [string]';\nCOMMENT ON COLUMN long_string.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN long_string.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN long_string.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\tid BIGINT,\n\t\"_meta/op\" TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE deletions IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/deletions';\nCOMMENT ON COLUMN deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\nINSERT INTO flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-redshift/materialize');" + "\nCREATE TABLE IF NOT EXISTS simple (\n\tid BIGINT,\n\tcanary TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE simple IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/simple';\nCOMMENT ON COLUMN simple.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN simple.canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE duplicate_keys_standard IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE duplicate_keys_delta IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\tid BIGINT,\n\tflow_published_at TIMESTAMPTZ,\n\tint BIGINT,\n\tstr TEXT\n);\n\nCOMMENT ON TABLE duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS multiple_types (\n\tid BIGINT,\n\tarray_int SUPER,\n\tbinary_field TEXT,\n\tbool_field BOOLEAN,\n\tfloat_field DOUBLE PRECISION,\n\tflow_published_at TIMESTAMPTZ,\n\tmultiple SUPER,\n\tnested SUPER,\n\tnullable_int BIGINT,\n\tstr_field TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE multiple_types IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN multiple_types.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN multiple_types.array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN multiple_types.binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN multiple_types.float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN multiple_types.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN multiple_types.nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN multiple_types.nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN multiple_types.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS formatted_strings (\n\tid BIGINT,\n\tdate DATE,\n\tdatetime TIMESTAMPTZ,\n\tflow_published_at TIMESTAMPTZ,\n\tint_and_str NUMERIC(38,0),\n\tint_str NUMERIC(38,0),\n\tnum_and_str DOUBLE PRECISION,\n\tnum_str DOUBLE PRECISION,\n\ttime TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE formatted_strings IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN formatted_strings.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN formatted_strings.date IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN formatted_strings.int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN formatted_strings.num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.time IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS long_string (\n\tid TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tstr_field TEXT,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE long_string IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/long-string';\nCOMMENT ON COLUMN long_string.id IS 'auto-generated projection of JSON at: /id with inferred types: [string]';\nCOMMENT ON COLUMN long_string.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN long_string.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN long_string.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\tid BIGINT,\n\t\"_meta/op\" TEXT,\n\tflow_published_at TIMESTAMPTZ,\n\tflow_document SUPER\n);\n\nCOMMENT ON TABLE deletions IS 'Generated for materialization tests/materialize-redshift/materialize of collection tests/deletions';\nCOMMENT ON COLUMN deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n" ] [ "connectorState", @@ -481,7 +481,7 @@ } [ "applied.actionDescription", - "UPDATE flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-redshift/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-snowflake/snapshot.json b/tests/materialize/materialize-snowflake/snapshot.json index bfbdc27ce7..c33d1ffa65 100644 --- a/tests/materialize/materialize-snowflake/snapshot.json +++ b/tests/materialize/materialize-snowflake/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nCREATE TABLE IF NOT EXISTS simple (\n\tid INTEGER NOT NULL,\n\tcanary TEXT NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE simple IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/simple';\nCOMMENT ON COLUMN simple.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN simple.canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE duplicate_keys_standard IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL\n);\n\nCOMMENT ON TABLE duplicate_keys_delta IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL\n);\n\nCOMMENT ON TABLE duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS multiple_types (\n\tid INTEGER NOT NULL,\n\tarray_int VARIANT,\n\tbinary_field TEXT,\n\tbool_field BOOLEAN,\n\tfloat_field FLOAT,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tmultiple VARIANT,\n\tnested VARIANT,\n\tnullable_int INTEGER,\n\tstr_field TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE multiple_types IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN multiple_types.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN multiple_types.array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN multiple_types.binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN multiple_types.float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN multiple_types.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN multiple_types.nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN multiple_types.nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN multiple_types.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS formatted_strings (\n\tid INTEGER NOT NULL,\n\tdate DATE,\n\tdatetime TIMESTAMP_LTZ,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint_and_str INTEGER,\n\tint_str INTEGER,\n\tnum_and_str FLOAT,\n\tnum_str FLOAT,\n\ttime TEXT,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE formatted_strings IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN formatted_strings.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN formatted_strings.date IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN formatted_strings.int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN formatted_strings.num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.time IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS symbols (\n\t\"testing (%s)\" TEXT NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tid TEXT,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (\"testing (%s)\")\n);\n\nCOMMENT ON TABLE symbols IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/symbols';\nCOMMENT ON COLUMN symbols.\"testing (%s)\" IS 'auto-generated projection of JSON at: /testing (%s) with inferred types: [string]';\nCOMMENT ON COLUMN symbols.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN symbols.id IS 'auto-generated projection of JSON at: /id with inferred types: [string]';\nCOMMENT ON COLUMN symbols.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS unsigned_bigint (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tunsigned_bigint INTEGER,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE unsigned_bigint IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/unsigned-bigint';\nCOMMENT ON COLUMN unsigned_bigint.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN unsigned_bigint.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN unsigned_bigint.unsigned_bigint IS 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]';\nCOMMENT ON COLUMN unsigned_bigint.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\tid INTEGER NOT NULL,\n\t\"_meta/op\" TEXT,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE deletions IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/deletions';\nCOMMENT ON COLUMN deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\nINSERT INTO ESTUARY_SCHEMA.flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-snowflake/materialize');" + "\nCREATE TABLE IF NOT EXISTS simple (\n\tid INTEGER NOT NULL,\n\tcanary TEXT NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE simple IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/simple';\nCOMMENT ON COLUMN simple.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN simple.canary IS 'auto-generated projection of JSON at: /canary with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN simple.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_standard (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE duplicate_keys_standard IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_standard.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_standard.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_standard.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL\n);\n\nCOMMENT ON TABLE duplicate_keys_delta IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS duplicate_keys_delta_exclude_flow_doc (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint INTEGER,\n\tstr TEXT NOT NULL\n);\n\nCOMMENT ON TABLE duplicate_keys_delta_exclude_flow_doc IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/duplicated-keys';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.int IS 'auto-generated projection of JSON at: /int with inferred types: [integer]';\nCOMMENT ON COLUMN duplicate_keys_delta_exclude_flow_doc.str IS 'auto-generated projection of JSON at: /str with inferred types: [string]';\n\n\nCREATE TABLE IF NOT EXISTS multiple_types (\n\tid INTEGER NOT NULL,\n\tarray_int VARIANT,\n\tbinary_field TEXT,\n\tbool_field BOOLEAN,\n\tfloat_field FLOAT,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tmultiple VARIANT,\n\tnested VARIANT,\n\tnullable_int INTEGER,\n\tstr_field TEXT NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE multiple_types IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/multiple-data-types';\nCOMMENT ON COLUMN multiple_types.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN multiple_types.array_int IS 'auto-generated projection of JSON at: /array_int with inferred types: [array]';\nCOMMENT ON COLUMN multiple_types.binary_field IS 'auto-generated projection of JSON at: /binary_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.bool_field IS 'auto-generated projection of JSON at: /bool_field with inferred types: [boolean]';\nCOMMENT ON COLUMN multiple_types.float_field IS 'auto-generated projection of JSON at: /float_field with inferred types: [number]';\nCOMMENT ON COLUMN multiple_types.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.multiple IS 'auto-generated projection of JSON at: /multiple with inferred types: [array boolean null number object string]';\nCOMMENT ON COLUMN multiple_types.nested IS 'auto-generated projection of JSON at: /nested with inferred types: [object]';\nCOMMENT ON COLUMN multiple_types.nullable_int IS 'auto-generated projection of JSON at: /nullable_int with inferred types: [integer null]';\nCOMMENT ON COLUMN multiple_types.str_field IS 'auto-generated projection of JSON at: /str_field with inferred types: [string]';\nCOMMENT ON COLUMN multiple_types.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS formatted_strings (\n\tid INTEGER NOT NULL,\n\tdate DATE,\n\tdatetime TIMESTAMP_LTZ,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tint_and_str INTEGER,\n\tint_str INTEGER,\n\tnum_and_str FLOAT,\n\tnum_str FLOAT,\n\ttime TEXT,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE formatted_strings IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/formatted-strings';\nCOMMENT ON COLUMN formatted_strings.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN formatted_strings.date IS 'auto-generated projection of JSON at: /date with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.datetime IS 'auto-generated projection of JSON at: /datetime with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.int_and_str IS 'auto-generated projection of JSON at: /int_and_str with inferred types: [integer string]';\nCOMMENT ON COLUMN formatted_strings.int_str IS 'auto-generated projection of JSON at: /int_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.num_and_str IS 'auto-generated projection of JSON at: /num_and_str with inferred types: [number string]';\nCOMMENT ON COLUMN formatted_strings.num_str IS 'auto-generated projection of JSON at: /num_str with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.time IS 'auto-generated projection of JSON at: /time with inferred types: [string]';\nCOMMENT ON COLUMN formatted_strings.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS symbols (\n\t\"testing (%s)\" TEXT NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tid TEXT,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (\"testing (%s)\")\n);\n\nCOMMENT ON TABLE symbols IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/symbols';\nCOMMENT ON COLUMN symbols.\"testing (%s)\" IS 'auto-generated projection of JSON at: /testing (%s) with inferred types: [string]';\nCOMMENT ON COLUMN symbols.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN symbols.id IS 'auto-generated projection of JSON at: /id with inferred types: [string]';\nCOMMENT ON COLUMN symbols.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS unsigned_bigint (\n\tid INTEGER NOT NULL,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tunsigned_bigint INTEGER,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE unsigned_bigint IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/unsigned-bigint';\nCOMMENT ON COLUMN unsigned_bigint.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN unsigned_bigint.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN unsigned_bigint.unsigned_bigint IS 'auto-generated projection of JSON at: /unsigned_bigint with inferred types: [integer]';\nCOMMENT ON COLUMN unsigned_bigint.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n\n\nCREATE TABLE IF NOT EXISTS deletions (\n\tid INTEGER NOT NULL,\n\t\"_meta/op\" TEXT,\n\tflow_published_at TIMESTAMP_LTZ NOT NULL,\n\tflow_document VARIANT NOT NULL,\n\n\tPRIMARY KEY (id)\n);\n\nCOMMENT ON TABLE deletions IS 'Generated for materialization tests/materialize-snowflake/materialize of collection tests/deletions';\nCOMMENT ON COLUMN deletions.id IS 'auto-generated projection of JSON at: /id with inferred types: [integer]';\nCOMMENT ON COLUMN deletions.\"_meta/op\" IS 'auto-generated projection of JSON at: /_meta/op with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_published_at IS 'Flow Publication Time\nFlow publication date-time of this document\nauto-generated projection of JSON at: /_meta/uuid with inferred types: [string]';\nCOMMENT ON COLUMN deletions.flow_document IS 'auto-generated projection of JSON at: with inferred types: [object]';\n" ] [ "connectorState", @@ -709,7 +709,7 @@ } [ "applied.actionDescription", - "UPDATE ESTUARY_SCHEMA.flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-snowflake/materialize';" + "" ] [ "connectorState", diff --git a/tests/materialize/materialize-sqlserver/snapshot.json b/tests/materialize/materialize-sqlserver/snapshot.json index b07f9d04ef..709447c172 100644 --- a/tests/materialize/materialize-sqlserver/snapshot.json +++ b/tests/materialize/materialize-sqlserver/snapshot.json @@ -1,6 +1,6 @@ [ "applied.actionDescription", - "\nIF OBJECT_ID(N'flow_materializations_v2', 'U') IS NULL BEGIN\nCREATE TABLE flow_materializations_v2 (\n\t\tmaterialization varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tversion varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tspec varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (materialization)\n);\nEND;\n\n\nIF OBJECT_ID(N'flow_checkpoints_v1', 'U') IS NULL BEGIN\nCREATE TABLE flow_checkpoints_v1 (\n\t\tmaterialization varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tkey_begin BIGINT NOT NULL,\n\t\tkey_end BIGINT NOT NULL,\n\t\tfence BIGINT NOT NULL,\n\t\t\"checkpoint\" varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (materialization, key_begin, key_end)\n);\nEND;\n\n\nIF OBJECT_ID(N'Simple', 'U') IS NULL BEGIN\nCREATE TABLE \"Simple\" (\n\t\tid BIGINT NOT NULL,\n\t\tcanary varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_standard', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_standard (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_delta', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_delta (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_delta_exclude_flow_doc', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL\n);\nEND;\n\n\nIF OBJECT_ID(N'Multiple Types', 'U') IS NULL BEGIN\nCREATE TABLE \"Multiple Types\" (\n\t\tid BIGINT NOT NULL,\n\t\tarray_int varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tbinary_field varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tbool_field BIT,\n\t\tfloat_field DOUBLE PRECISION,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tmultiple varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tnested varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tnullable_int BIGINT,\n\t\tstr_field varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'Formatted Strings', 'U') IS NULL BEGIN\nCREATE TABLE \"Formatted Strings\" (\n\t\tid BIGINT NOT NULL,\n\t\t\"date\" DATE,\n\t\tdatetime DATETIME2,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tint_and_str BIGINT,\n\t\tint_str BIGINT,\n\t\tnum_and_str DOUBLE PRECISION,\n\t\tnum_str DOUBLE PRECISION,\n\t\t\"time\" TIME,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'compound_key', 'U') IS NULL BEGIN\nCREATE TABLE compound_key (\n\t\tyin varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tyan varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tnum BIGINT NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (yin, yan)\n);\nEND;\n\n\nIF OBJECT_ID(N'unsigned_bigint', 'U') IS NULL BEGIN\nCREATE TABLE unsigned_bigint (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tunsigned_bigint varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'deletions', 'U') IS NULL BEGIN\nCREATE TABLE deletions (\n\t\tid BIGINT NOT NULL,\n\t\t\"_meta/op\" varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\nINSERT INTO flow_materializations_v2 (version, spec, materialization) VALUES ('ffffffffffffffff', '(a-base64-encoded-value)', 'tests/materialize-sqlserver/materialize');" + "\nIF OBJECT_ID(N'Simple', 'U') IS NULL BEGIN\nCREATE TABLE \"Simple\" (\n\t\tid BIGINT NOT NULL,\n\t\tcanary varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_standard', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_standard (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_delta', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_delta (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL\n);\nEND;\n\n\nIF OBJECT_ID(N'duplicate_keys_delta_exclude_flow_doc', 'U') IS NULL BEGIN\nCREATE TABLE duplicate_keys_delta_exclude_flow_doc (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\t\"int\" BIGINT,\n\t\tstr varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL\n);\nEND;\n\n\nIF OBJECT_ID(N'Multiple Types', 'U') IS NULL BEGIN\nCREATE TABLE \"Multiple Types\" (\n\t\tid BIGINT NOT NULL,\n\t\tarray_int varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tbinary_field varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tbool_field BIT,\n\t\tfloat_field DOUBLE PRECISION,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tmultiple varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tnested varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tnullable_int BIGINT,\n\t\tstr_field varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'Formatted Strings', 'U') IS NULL BEGIN\nCREATE TABLE \"Formatted Strings\" (\n\t\tid BIGINT NOT NULL,\n\t\t\"date\" DATE,\n\t\tdatetime DATETIME2,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tint_and_str BIGINT,\n\t\tint_str BIGINT,\n\t\tnum_and_str DOUBLE PRECISION,\n\t\tnum_str DOUBLE PRECISION,\n\t\t\"time\" TIME,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'compound_key', 'U') IS NULL BEGIN\nCREATE TABLE compound_key (\n\t\tyin varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tyan varchar(900) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tnum BIGINT NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (yin, yan)\n);\nEND;\n\n\nIF OBJECT_ID(N'unsigned_bigint', 'U') IS NULL BEGIN\nCREATE TABLE unsigned_bigint (\n\t\tid BIGINT NOT NULL,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tunsigned_bigint varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n\n\nIF OBJECT_ID(N'deletions', 'U') IS NULL BEGIN\nCREATE TABLE deletions (\n\t\tid BIGINT NOT NULL,\n\t\t\"_meta/op\" varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8,\n\t\tflow_published_at DATETIME2 NOT NULL,\n\t\tflow_document varchar(MAX) COLLATE Latin1_General_100_BIN2_UTF8 NOT NULL,\n\n\t\tPRIMARY KEY (id)\n);\nEND;\n" ] [ "connectorState", @@ -510,7 +510,7 @@ } [ "applied.actionDescription", - "UPDATE flow_materializations_v2 SET version = 'ffffffffffffffff', spec = '(a-base64-encoded-value)' WHERE materialization = 'tests/materialize-sqlserver/materialize';" + "" ] [ "connectorState",