Skip to content

Commit

Permalink
Merge branch 'main' into cdc-parallel-sync-normalize
Browse files Browse the repository at this point in the history
  • Loading branch information
serprex authored Dec 5, 2023
2 parents ed6e356 + 11197a0 commit 2eb3046
Show file tree
Hide file tree
Showing 31 changed files with 2,407 additions and 1,876 deletions.
1 change: 1 addition & 0 deletions dev-peerdb.sh
Original file line number Diff line number Diff line change
Expand Up @@ -7,5 +7,6 @@ then
exit 1
fi

export PEERDB_VERSION_SHA_SHORT=local-$(git rev-parse --short HEAD)
docker compose -f docker-compose-dev.yml up --build \
--no-attach temporal --no-attach pyroscope --no-attach temporal-ui
3 changes: 3 additions & 0 deletions docker-bake.hcl
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,9 @@ target "flow-api" {
"linux/amd64",
"linux/arm64",
]
args = {
PEERDB_VERSION_SHA_SHORT = "${SHA_SHORT}"
}
tags = [
"${REGISTRY}/flow-api:${TAG}",
"${REGISTRY}/flow-api:${SHA_SHORT}",
Expand Down
4 changes: 3 additions & 1 deletion docker-compose-dev.yml
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ x-catalog-config: &catalog-config
PEERDB_CATALOG_DATABASE: postgres

x-flow-worker-env: &flow-worker-env
# For Temporal Cloud, this will look like:
# For Temporal Cloud, this will look like:
# <yournamespace>.<id>.tmprl.cloud:7233
TEMPORAL_HOST_PORT: temporal:7233
PEERDB_TEMPORAL_NAMESPACE: default
Expand Down Expand Up @@ -115,6 +115,8 @@ services:
context: .
dockerfile: stacks/flow.Dockerfile
target: flow-api
args:
PEERDB_VERSION_SHA_SHORT: ${PEERDB_VERSION_SHA_SHORT:-}
ports:
- 8112:8112
- 8113:8113
Expand Down
6 changes: 0 additions & 6 deletions flow/activities/flowable.go
Original file line number Diff line number Diff line change
Expand Up @@ -763,12 +763,6 @@ func (a *FlowableActivity) QRepWaitUntilNewRows(ctx context.Context,
waitBetweenBatches = time.Duration(config.WaitBetweenBatchesSeconds) * time.Second
}

if config.WatermarkColumn == "xmin" {
// for xmin we ignore the wait between batches, as seq scan time is
// extremely slow.
waitBetweenBatches = 10 * time.Second
}

srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer)
if err != nil {
return fmt.Errorf("failed to get qrep source connector: %w", err)
Expand Down
7 changes: 3 additions & 4 deletions flow/cmd/handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,10 @@ func (h *FlowRequestHandler) getPeerID(ctx context.Context, peerName string) (in
}

func schemaForTableIdentifier(tableIdentifier string, peerDBType int32) string {
tableIdentifierParts := strings.Split(tableIdentifier, ".")
if len(tableIdentifierParts) == 1 && peerDBType != int32(protos.DBType_BIGQUERY) {
tableIdentifierParts = append([]string{"public"}, tableIdentifierParts...)
if peerDBType != int32(protos.DBType_BIGQUERY) && !strings.ContainsRune(tableIdentifier, '.') {
return "public." + tableIdentifier
}
return strings.Join(tableIdentifierParts, ".")
return tableIdentifier
}

func (h *FlowRequestHandler) createCdcJobEntry(ctx context.Context,
Expand Down
4 changes: 2 additions & 2 deletions flow/connectors/bigquery/merge_statement_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,12 +172,12 @@ and updating the other columns (not the unchanged toast columns)
7. Return the list of generated update statements.
*/
func (m *mergeStmtGenerator) generateUpdateStatements(allCols []string, unchangedToastCols []string) []string {
updateStmts := make([]string, 0)
updateStmts := make([]string, 0, len(unchangedToastCols))

for _, cols := range unchangedToastCols {
unchangedColsArray := strings.Split(cols, ", ")
otherCols := utils.ArrayMinus(allCols, unchangedColsArray)
tmpArray := make([]string, 0)
tmpArray := make([]string, 0, len(otherCols))
for _, colName := range otherCols {
tmpArray = append(tmpArray, fmt.Sprintf("`%s` = _peerdb_deduped.%s", colName, colName))
}
Expand Down
4 changes: 2 additions & 2 deletions flow/connectors/bigquery/qrep_avro_sync.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@ type AvroSchema struct {
func DefineAvroSchema(dstTableName string,
dstTableMetadata *bigquery.TableMetadata) (*model.QRecordAvroSchemaDefinition, error) {
avroFields := []AvroField{}
nullableFields := map[string]bool{}
nullableFields := make(map[string]struct{})

for _, bqField := range dstTableMetadata.Schema {
avroType, err := GetAvroType(bqField)
Expand All @@ -197,7 +197,7 @@ func DefineAvroSchema(dstTableName string,
// If a field is nullable, its Avro type should be ["null", actualType]
if !bqField.Required {
avroType = []interface{}{"null", avroType}
nullableFields[bqField.Name] = true
nullableFields[bqField.Name] = struct{}{}
}

avroFields = append(avroFields, AvroField{
Expand Down
9 changes: 5 additions & 4 deletions flow/connectors/postgres/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -572,15 +572,16 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st
}

func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedToastColsLists []string) string {
updateStmts := make([]string, 0)
updateStmts := make([]string, 0, len(unchangedToastColsLists))

for _, cols := range unchangedToastColsLists {
unchangedColsArray := make([]string, 0)
for _, unchangedToastCol := range strings.Split(cols, ",") {
unquotedUnchangedColsArray := strings.Split(cols, ",")
unchangedColsArray := make([]string, 0, len(unquotedUnchangedColsArray))
for _, unchangedToastCol := range unquotedUnchangedColsArray {
unchangedColsArray = append(unchangedColsArray, fmt.Sprintf(`"%s"`, unchangedToastCol))
}
otherCols := utils.ArrayMinus(allCols, unchangedColsArray)
tmpArray := make([]string, 0)
tmpArray := make([]string, 0, len(otherCols))
for _, colName := range otherCols {
tmpArray = append(tmpArray, fmt.Sprintf("%s=src.%s", colName, colName))
}
Expand Down
9 changes: 2 additions & 7 deletions flow/connectors/postgres/qrep.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,11 +274,6 @@ func (c *PostgresConnector) getMinMaxValues(

func (c *PostgresConnector) CheckForUpdatedMaxValue(config *protos.QRepConfig,
last *protos.QRepPartition) (bool, error) {
// for xmin lets always assume there are updates
if config.WatermarkColumn == "xmin" {
return true, nil
}

tx, err := c.pool.Begin(c.ctx)
if err != nil {
return false, fmt.Errorf("unable to begin transaction for getting max value: %w", err)
Expand Down Expand Up @@ -571,9 +566,9 @@ func (c *PostgresConnector) PullXminRecordStream(

var numRecords int
if partition.Range != nil {
numRecords, currentSnapshotXmin, err = executor.ExecuteAndProcessQueryStreamGettingCurrentTxid(stream, query, oldxid)
numRecords, currentSnapshotXmin, err = executor.ExecuteAndProcessQueryStreamGettingCurrentSnapshotXmin(stream, query, oldxid)
} else {
numRecords, currentSnapshotXmin, err = executor.ExecuteAndProcessQueryStreamGettingCurrentTxid(stream, query)
numRecords, currentSnapshotXmin, err = executor.ExecuteAndProcessQueryStreamGettingCurrentSnapshotXmin(stream, query)
}
if err != nil {
return 0, currentSnapshotXmin, err
Expand Down
2 changes: 1 addition & 1 deletion flow/connectors/postgres/qrep_query_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream(
return totalRecordsFetched, err
}

func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamGettingCurrentTxid(
func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamGettingCurrentSnapshotXmin(
stream *model.QRecordStream,
query string,
args ...interface{},
Expand Down
13 changes: 5 additions & 8 deletions flow/connectors/snowflake/snowflake.go
Original file line number Diff line number Diff line change
Expand Up @@ -926,15 +926,12 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement(

// parseTableName parses a table name into schema and table name.
func parseTableName(tableName string) (*tableNameComponents, error) {
parts := strings.Split(tableName, ".")
if len(parts) != 2 {
schemaIdentifier, tableIdentifier, hasDot := strings.Cut(tableName, ".")
if !hasDot || strings.ContainsRune(tableIdentifier, '.') {
return nil, fmt.Errorf("invalid table name: %s", tableName)
}

return &tableNameComponents{
schemaIdentifier: parts[0],
tableIdentifier: parts[1],
}, nil
return &tableNameComponents{schemaIdentifier, tableIdentifier}, nil
}

func (c *SnowflakeConnector) jobMetadataExists(jobName string) (bool, error) {
Expand Down Expand Up @@ -1032,12 +1029,12 @@ and updating the other columns.
func (c *SnowflakeConnector) generateUpdateStatements(
syncedAtCol string, softDeleteCol string, softDelete bool,
allCols []string, unchangedToastCols []string) []string {
updateStmts := make([]string, 0)
updateStmts := make([]string, 0, len(unchangedToastCols))

for _, cols := range unchangedToastCols {
unchangedColsArray := strings.Split(cols, ",")
otherCols := utils.ArrayMinus(allCols, unchangedColsArray)
tmpArray := make([]string, 0)
tmpArray := make([]string, 0, len(otherCols)+2)
for _, colName := range otherCols {
quotedUpperColName := fmt.Sprintf(`"%s"`, strings.ToUpper(colName))
tmpArray = append(tmpArray, fmt.Sprintf("%s = SOURCE.%s", quotedUpperColName, quotedUpperColName))
Expand Down
7 changes: 4 additions & 3 deletions flow/connectors/utils/array.go
Original file line number Diff line number Diff line change
@@ -1,15 +1,16 @@
package utils

func ArrayMinus(first []string, second []string) []string {
lookup := make(map[string]bool)
lookup := make(map[string]struct{})
// Add elements from arrayB to the lookup map
for _, element := range second {
lookup[element] = true
lookup[element] = struct{}{}
}
// Iterate over arrayA and check if the element is present in the lookup map
var result []string
for _, element := range first {
if !lookup[element] {
_, exists := lookup[element]
if !exists {
result = append(result, element)
}
}
Expand Down
2 changes: 1 addition & 1 deletion flow/connectors/utils/avro/avro_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func (p *PeerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) (
avroConverter := model.NewQRecordAvroConverter(
qRecord,
p.targetDWH,
&p.avroSchema.NullableFields,
p.avroSchema.NullableFields,
colNames,
)

Expand Down
11 changes: 2 additions & 9 deletions flow/connectors/utils/aws.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,18 +81,11 @@ func NewS3BucketAndPrefix(s3Path string) (*S3BucketAndPrefix, error) {
stagingPath := strings.TrimPrefix(s3Path, "s3://")

// Split into bucket and prefix
splitPath := strings.SplitN(stagingPath, "/", 2)

bucket := splitPath[0]
prefix := ""
if len(splitPath) > 1 {
// Remove leading and trailing slashes from prefix
prefix = strings.Trim(splitPath[1], "/")
}
bucket, prefix, _ := strings.Cut(stagingPath, "/")

return &S3BucketAndPrefix{
Bucket: bucket,
Prefix: prefix,
Prefix: strings.Trim(prefix, "/"),
}, nil
}

Expand Down
2 changes: 1 addition & 1 deletion flow/e2e/snowflake/qrep_flow_sf_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() {
qrepConfig.WatermarkColumn = "xmin"
s.NoError(err)

e2e.RunQrepFlowWorkflow(env, qrepConfig)
e2e.RunXminFlowWorkflow(env, qrepConfig)

// Verify workflow completes without error
s.True(env.IsWorkflowCompleted())
Expand Down
10 changes: 9 additions & 1 deletion flow/e2e/test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import (
"go.temporal.io/sdk/testsuite"
)

// readFileToBytes reads a file to a byte array.
// ReadFileToBytes reads a file to a byte array.
func ReadFileToBytes(path string) ([]byte, error) {
var ret []byte

Expand Down Expand Up @@ -49,6 +49,7 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) {
env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow)
env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow)
env.RegisterWorkflow(peerflow.QRepFlowWorkflow)
env.RegisterWorkflow(peerflow.XminFlowWorkflow)
env.RegisterWorkflow(peerflow.QRepPartitionWorkflow)
env.RegisterActivity(&activities.FlowableActivity{})
env.RegisterActivity(&activities.SnapshotActivity{})
Expand Down Expand Up @@ -303,6 +304,13 @@ func RunQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.
env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, state)
}

func RunXminFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) {
state := peerflow.NewQRepFlowState()
state.LastPartition.PartitionId = uuid.New().String()
time.Sleep(5 * time.Second)
env.ExecuteWorkflow(peerflow.XminFlowWorkflow, config, state)
}

func GetOwnersSchema() *model.QRecordSchema {
return &model.QRecordSchema{
Fields: []*model.QField{
Expand Down
Loading

0 comments on commit 2eb3046

Please sign in to comment.