From cf417d039b0ead8ca5ede5bba77e936f7af95544 Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Sat, 30 Sep 2023 05:35:46 +0530 Subject: [PATCH 1/7] chore: introduce repo for sources job --- services/notifier/notifier.go | 2 +- warehouse/http.go | 6 +- warehouse/http_test.go | 8 +- warehouse/internal/errors/errors.go | 2 - warehouse/internal/model/source.go | 42 +++ warehouse/internal/model/upload.go | 5 +- warehouse/internal/repo/source.go | 288 ++++++++++++++++ warehouse/internal/repo/source_test.go | 126 +++++++ warehouse/internal/repo/table_upload.go | 170 ++++++---- warehouse/jobs/http.go | 157 --------- warehouse/jobs/jobs.go | 64 ---- warehouse/jobs/runner.go | 425 ------------------------ warehouse/jobs/types.go | 89 ----- warehouse/jobs/utils.go | 41 --- warehouse/slave_worker.go | 20 +- warehouse/slave_worker_test.go | 23 +- warehouse/source/http.go | 208 ++++++++++++ warehouse/{jobs => source}/http_test.go | 30 +- warehouse/source/source.go | 224 +++++++++++++ warehouse/warehouse.go | 12 +- 20 files changed, 1054 insertions(+), 888 deletions(-) create mode 100644 warehouse/internal/model/source.go create mode 100644 warehouse/internal/repo/source.go create mode 100644 warehouse/internal/repo/source_test.go delete mode 100644 warehouse/jobs/http.go delete mode 100644 warehouse/jobs/jobs.go delete mode 100644 warehouse/jobs/runner.go delete mode 100644 warehouse/jobs/types.go delete mode 100644 warehouse/jobs/utils.go create mode 100644 warehouse/source/http.go rename warehouse/{jobs => source}/http_test.go (95%) create mode 100644 warehouse/source/source.go diff --git a/services/notifier/notifier.go b/services/notifier/notifier.go index 6f7dee9469..0353cdcdd4 100644 --- a/services/notifier/notifier.go +++ b/services/notifier/notifier.go @@ -39,7 +39,7 @@ const ( JobTypeAsync JobType = "async_job" ) -type JobStatus string +type JobStatus = string const ( Waiting JobStatus = "waiting" diff --git a/warehouse/http.go b/warehouse/http.go index 23b4a27923..8a8dcf5757 100644 --- a/warehouse/http.go +++ b/warehouse/http.go @@ -30,8 +30,8 @@ import ( sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" "github.com/rudderlabs/rudder-server/warehouse/internal/model" "github.com/rudderlabs/rudder-server/warehouse/internal/repo" - "github.com/rudderlabs/rudder-server/warehouse/jobs" "github.com/rudderlabs/rudder-server/warehouse/multitenant" + "github.com/rudderlabs/rudder-server/warehouse/source" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" ) @@ -71,7 +71,7 @@ type Api struct { bcConfig backendconfig.BackendConfig tenantManager *multitenant.Manager bcManager *backendConfigManager - asyncManager *jobs.AsyncJobWh + asyncManager *source.Manager stagingRepo *repo.StagingFiles uploadRepo *repo.Uploads schemaRepo *repo.WHSchema @@ -96,7 +96,7 @@ func NewApi( notifier *notifier.Notifier, tenantManager *multitenant.Manager, bcManager *backendConfigManager, - asyncManager *jobs.AsyncJobWh, + asyncManager *source.Manager, triggerStore *sync.Map, ) *Api { a := &Api{ diff --git a/warehouse/http_test.go b/warehouse/http_test.go index 168ad4df96..d3353549c9 100644 --- a/warehouse/http_test.go +++ b/warehouse/http_test.go @@ -21,8 +21,8 @@ import ( kithelper "github.com/rudderlabs/rudder-go-kit/testhelper" sqlmiddleware "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" - "github.com/rudderlabs/rudder-server/warehouse/jobs" "github.com/rudderlabs/rudder-server/warehouse/multitenant" + "github.com/rudderlabs/rudder-server/warehouse/source" "github.com/golang/mock/gomock" "github.com/ory/dockertest/v3" @@ -183,12 +183,12 @@ func TestHTTPApi(t *testing.T) { err = n.Setup(ctx, pgResource.DBDsn) require.NoError(t, err) - sourcesManager := jobs.New( + sourcesManager := source.New( ctx, db, n, ) - jobs.WithConfig(sourcesManager, config.Default) + source.WithConfig(sourcesManager, config.Default) g, gCtx := errgroup.WithContext(ctx) g.Go(func() error { @@ -200,7 +200,7 @@ func TestHTTPApi(t *testing.T) { return nil }) g.Go(func() error { - return sourcesManager.Run() + return sourcesManager.Run(gCtx) }) setupCh := make(chan struct{}) diff --git a/warehouse/internal/errors/errors.go b/warehouse/internal/errors/errors.go index 8edf2d57e8..0fa4ecec9d 100644 --- a/warehouse/internal/errors/errors.go +++ b/warehouse/internal/errors/errors.go @@ -9,6 +9,4 @@ var ( ErrNoWarehouseFound = errors.New("no warehouse found") ErrWorkspaceFromSourceNotFound = errors.New("workspace from source not found") ErrMarshallResponse = errors.New("can't marshall response") - ErrInvalidRequest = errors.New("invalid request") - ErrJobsApiNotInitialized = errors.New("warehouse jobs api not initialized") ) diff --git a/warehouse/internal/model/source.go b/warehouse/internal/model/source.go new file mode 100644 index 0000000000..8018b2e5aa --- /dev/null +++ b/warehouse/internal/model/source.go @@ -0,0 +1,42 @@ +package model + +import ( + "encoding/json" + "time" +) + +type SourceJobType = string + +const ( + DeleteByJobRunID SourceJobType = "deletebyjobrunid" +) + +type SourceJobStatus = string + +const ( + SourceJobStatusWaiting SourceJobStatus = "waiting" + SourceJobStatusExecuting SourceJobStatus = "executing" + SourceJobStatusSucceeded SourceJobStatus = "succeeded" + SourceJobStatusAborted SourceJobStatus = "aborted" + SourceJobStatusFailed SourceJobStatus = "failed" +) + +type SourceJob struct { + ID int64 + + SourceID string + DestinationID string + WorkspaceID string + + TableName string + + Status SourceJobStatus + Error error + JobType SourceJobType + + Metadata json.RawMessage + Attempts int64 + + CreatedAt time.Time + UpdatedAt time.Time +} diff --git a/warehouse/internal/model/upload.go b/warehouse/internal/model/upload.go index 9c54d397f2..d803488864 100644 --- a/warehouse/internal/model/upload.go +++ b/warehouse/internal/model/upload.go @@ -37,8 +37,9 @@ const ( ) var ( - ErrUploadNotFound = errors.New("upload not found") - ErrNoUploadsFound = errors.New("no uploads found") + ErrUploadNotFound = errors.New("upload not found") + ErrSourcesJobNotFound = errors.New("sources job not found") + ErrNoUploadsFound = errors.New("no uploads found") ) type Upload struct { diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go new file mode 100644 index 0000000000..b72478f64f --- /dev/null +++ b/warehouse/internal/repo/source.go @@ -0,0 +1,288 @@ +package repo + +import ( + "context" + "database/sql" + "errors" + "fmt" + + "github.com/rudderlabs/rudder-server/utils/timeutil" + sqlmiddleware "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" + "github.com/rudderlabs/rudder-server/warehouse/internal/model" + warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" +) + +const ( + sourceJobTableName = warehouseutils.WarehouseAsyncJobTable + sourceJobColumns = ` + id, + source_id, + destination_id, + status, + created_at, + updated_at, + tablename, + error, + async_job_type, + metadata, + attempt, + workspace_id + ` +) + +type Source repo + +func NewSource(db *sqlmiddleware.DB, opts ...Opt) *Source { + r := &Source{ + db: db, + now: timeutil.Now, + } + for _, opt := range opts { + opt((*repo)(r)) + } + return r +} + +func (repo *Source) Insert( + ctx context.Context, + sourceJobs []model.SourceJob, +) ([]int64, error) { + txn, err := repo.db.BeginTx(ctx, &sql.TxOptions{}) + if err != nil { + return nil, fmt.Errorf(`begin transaction: %w`, err) + } + defer func() { + if err != nil { + _ = txn.Rollback() + } + }() + + stmt, err := txn.PrepareContext( + ctx, ` + INSERT INTO `+sourceJobTableName+` ( + source_id, destination_id, tablename, + status, created_at, updated_at, async_job_type, + workspace_id, metadata + ) + VALUES + ($1, $2, $3, $4, $5, $6, $7, $8, $9) RETURNING id; +`, + ) + if err != nil { + return nil, fmt.Errorf(`preparing statement: %w`, err) + } + defer func() { _ = stmt.Close() }() + + var ids []int64 + for _, sourceJob := range sourceJobs { + var id int64 + err = stmt.QueryRowContext( + ctx, + sourceJob.SourceID, + sourceJob.DestinationID, + sourceJob.TableName, + model.SourceJobStatusWaiting, + repo.now(), + repo.now(), + sourceJob.Status, + sourceJob.WorkspaceID, + sourceJob.Metadata, + ).Scan(&id) + if err != nil { + return nil, fmt.Errorf(`executing: %w`, err) + } + + ids = append(ids, id) + } + + if err = txn.Commit(); err != nil { + return nil, fmt.Errorf(`commiting: %w`, err) + } + + return ids, nil +} + +func (repo *Source) Reset( + ctx context.Context, +) error { + _, err := repo.db.ExecContext(ctx, ` + UPDATE + `+sourceJobTableName+` + SET + status = $1 + WHERE + status = $2 OR status = $3; + `, + model.SourceJobStatusWaiting, + model.SourceJobStatusExecuting, + model.SourceJobStatusFailed, + ) + if err != nil { + return fmt.Errorf("resetting: %w", err) + } + return nil +} + +func (repo *Source) GetToProcess( + ctx context.Context, + limit int64, +) ([]model.SourceJob, error) { + rows, err := repo.db.QueryContext(ctx, ` + SELECT + `+sourceJobColumns+` + FROM + `+sourceJobTableName+` + WHERE + status = $1 OR status = $2 + LIMIT $3; + `, + model.SourceJobStatusWaiting, + model.SourceJobStatusFailed, + limit, + ) + if err != nil { + return nil, fmt.Errorf("querying: %w", err) + } + defer func() { _ = rows.Close() }() + + var sourceJobs []model.SourceJob + for rows.Next() { + var sourceJob model.SourceJob + err := repo.scanSourceJob(rows.Scan, &sourceJob) + if err != nil { + return nil, fmt.Errorf("scanning: %w", err) + } + + sourceJobs = append(sourceJobs, sourceJob) + } + + return sourceJobs, nil +} + +func (repo *Source) scanSourceJob( + scan scanFn, + sourceJob *model.SourceJob, +) error { + var errorRaw sql.NullString + var jobTypeRaw sql.NullString + + if err := scan( + &sourceJob.ID, + &sourceJob.SourceID, + &sourceJob.DestinationID, + &sourceJob.Status, + &sourceJob.CreatedAt, + &sourceJob.UpdatedAt, + &sourceJob.TableName, + &errorRaw, + &jobTypeRaw, + &sourceJob.Metadata, + &sourceJob.Attempts, + &sourceJob.WorkspaceID, + ); err != nil { + return fmt.Errorf("scanning: %w", err) + } + + if errorRaw.Valid { + sourceJob.Error = errors.New(errorRaw.String) + } + if jobTypeRaw.Valid { + switch jobTypeRaw.String { + case model.DeleteByJobRunID: + sourceJob.JobType = jobTypeRaw.String + default: + return fmt.Errorf("scanning: unknown job type: %s", jobTypeRaw.String) + } + } else { + return fmt.Errorf("scanning: job type is null") + } + + sourceJob.CreatedAt = sourceJob.CreatedAt.UTC() + sourceJob.UpdatedAt = sourceJob.UpdatedAt.UTC() + + return nil +} + +func (repo *Source) GetByJobRunTaskRun( + ctx context.Context, + jobRunID string, + taskRunID string, +) (*model.SourceJob, error) { + row := repo.db.QueryRowContext(ctx, ` + SELECT + `+sourceJobColumns+` + FROM + `+sourceJobTableName+` + WHERE + metadata->>'job_run_id' = $1 AND + metadata->>'task_run_id' = $2 + LIMIT 1; + `, + jobRunID, + taskRunID, + ) + + var sourceJob model.SourceJob + err := repo.scanSourceJob(row.Scan, &sourceJob) + if errors.Is(err, sql.ErrNoRows) { + return nil, model.ErrSourcesJobNotFound + } + if err != nil { + return nil, fmt.Errorf("scanning: %w", err) + } + return &sourceJob, nil +} + +func (repo *Source) OnUpdateSuccess( + ctx context.Context, + id int64, +) error { + _, err := repo.db.ExecContext(ctx, ` + UPDATE + `+sourceJobTableName+` + SET + status = $1, + updated_at = $2 + WHERE + id = $3; +`, + model.SourceJobStatusSucceeded, + repo.now(), + id, + ) + if err != nil { + return fmt.Errorf("on update success: %w", err) + } + return nil +} + +func (repo *Source) OnUpdateFailure( + ctx context.Context, + id int64, + error error, + maxAttempt int, +) error { + _, err := repo.db.ExecContext(ctx, ` + UPDATE + `+sourceJobTableName+` + SET + status =( + CASE WHEN attempt > $1 THEN `+model.SourceJobStatusAborted+` + ELSE `+model.SourceJobStatusFailed+`) END + ), + attempt = attempt + 1, + updated_at = $2, + error = $3 + WHERE + id = $3; +`, + maxAttempt, + repo.now(), + error.Error(), + id, + ) + if err != nil { + return fmt.Errorf("on update success: %w", err) + } + return nil +} diff --git a/warehouse/internal/repo/source_test.go b/warehouse/internal/repo/source_test.go new file mode 100644 index 0000000000..74084dbbe5 --- /dev/null +++ b/warehouse/internal/repo/source_test.go @@ -0,0 +1,126 @@ +package repo_test + +import ( + "context" + "encoding/json" + warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" + "strconv" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/rudderlabs/rudder-server/warehouse/internal/model" + "github.com/rudderlabs/rudder-server/warehouse/internal/repo" +) + +func sourcesJobs( + sourceID, destinationID, workspaceID string, + count int, +) []model.SourceJob { + sourcesJobs := make([]model.SourceJob, 0, count) + for i := 0; i < count; i++ { + sourcesJobs = append(sourcesJobs, model.SourceJob{ + SourceID: sourceID, + DestinationID: destinationID, + TableName: "table" + strconv.Itoa(i), + WorkspaceID: workspaceID, + Metadata: json.RawMessage(`{"key": "value"}`), + }) + } + return sourcesJobs +} + +func TestSource_Insert(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, 10), + ) + require.NoError(t, err) + require.Len(t, ids, 10) + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + ids, err := repoSource.Insert(ctx, + sourcesJobs(sourceId, destinationId, workspaceId, 1), + ) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, ids) + }) +} + +func TestSource_Reset(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert(ctx, + sourcesJobs(sourceId, destinationId, workspaceId, 10), + ) + require.NoError(t, err) + require.Len(t, ids, 10) + + for _, id := range ids[0:3] { + _, err = db.ExecContext(ctx, `UPDATE `+warehouseutils.WarehouseAsyncJobTable+` SET status = $1 WHERE id = $2`, model.SourceJobStatusSucceeded, id) + require.NoError(t, err) + } + for _, id := range ids[3:6] { + _, err = db.ExecContext(ctx, `UPDATE `+warehouseutils.WarehouseAsyncJobTable+` SET status = $1 WHERE id = $2`, model.SourceJobStatusExecuting, id) + require.NoError(t, err) + } + for _, id := range ids[6:10] { + _, err = db.ExecContext(ctx, `UPDATE `+warehouseutils.WarehouseAsyncJobTable+` SET status = $1 WHERE id = $2`, model.SourceJobStatusFailed, id) + require.NoError(t, err) + } + + err = repoSource.Reset(ctx) + require.NoError(t, err) + + for _, id := range ids[0:3] { + var status string + err = db.QueryRowContext(ctx, `SELECT status FROM `+warehouseutils.WarehouseAsyncJobTable+` WHERE id = $1`, id).Scan(&status) + require.NoError(t, err) + require.Equal(t, model.SourceJobStatusSucceeded, status) + } + + for _, id := range ids[3:10] { + var status string + err = db.QueryRowContext(ctx, `SELECT status FROM `+warehouseutils.WarehouseAsyncJobTable+` WHERE id = $1`, id).Scan(&status) + require.NoError(t, err) + require.Equal(t, model.SourceJobStatusWaiting, status) + } + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + err := repoSource.Reset(ctx) + require.ErrorIs(t, err, context.Canceled) + }) +} diff --git a/warehouse/internal/repo/table_upload.go b/warehouse/internal/repo/table_upload.go index f6f5cca773..80fcc820b2 100644 --- a/warehouse/internal/repo/table_upload.go +++ b/warehouse/internal/repo/table_upload.go @@ -91,7 +91,15 @@ func (repo *TableUploads) Insert(ctx context.Context, uploadID int64, tableNames defer func() { _ = stmt.Close() }() for _, tableName := range tableNames { - _, err = stmt.ExecContext(ctx, uploadID, tableName, model.TableUploadWaiting, "{}", repo.now(), repo.now()) + _, err = stmt.ExecContext( + ctx, + uploadID, + tableName, + model.TableUploadWaiting, + "{}", + repo.now(), + repo.now(), + ) if err != nil { return fmt.Errorf(`stmt exec: %w`, err) } @@ -113,7 +121,20 @@ func (repo *TableUploads) GetByUploadID(ctx context.Context, uploadID int64) ([] return nil, fmt.Errorf("querying table uploads: %w", err) } - return repo.parseRows(rows) + var tableUploads []model.TableUpload + for rows.Next() { + var tableUpload model.TableUpload + err := scanTableUpload(rows.Scan, &tableUpload) + if err != nil { + return nil, fmt.Errorf("parsing rows: %w", err) + } + tableUploads = append(tableUploads, tableUpload) + } + if err := rows.Err(); err != nil { + return nil, err + } + + return tableUploads, nil } func (repo *TableUploads) GetByUploadIDAndTableName(ctx context.Context, uploadID int64, tableName string) (model.TableUpload, error) { @@ -124,71 +145,55 @@ func (repo *TableUploads) GetByUploadIDAndTableName(ctx context.Context, uploadI LIMIT 1; ` - rows, err := repo.db.QueryContext(ctx, query, uploadID, tableName) - if err != nil { - return model.TableUpload{}, fmt.Errorf("querying table uploads: %w", err) - } + row := repo.db.QueryRowContext(ctx, query, uploadID, tableName) - entries, err := repo.parseRows(rows) - if err != nil { - return model.TableUpload{}, fmt.Errorf("parsing rows: %w", err) + var tableUpload model.TableUpload + err := scanTableUpload(row.Scan, &tableUpload) + if errors.Is(err, sql.ErrNoRows) { + return tableUpload, fmt.Errorf("no table upload found with uploadID: %d, tableName: %s", uploadID, tableName) } - if len(entries) == 0 { - return model.TableUpload{}, fmt.Errorf("no table upload found with uploadID: %d, tableName: %s", uploadID, tableName) + if err != nil { + return tableUpload, fmt.Errorf("parsing rows: %w", err) } - return entries[0], err + return tableUpload, err } -func (*TableUploads) parseRows(rows *sqlmiddleware.Rows) ([]model.TableUpload, error) { - var tableUploads []model.TableUpload - - defer func() { _ = rows.Close() }() - - for rows.Next() { - var ( - tableUpload model.TableUpload - locationRaw sql.NullString - lastExecTimeRaw sql.NullTime - totalEvents sql.NullInt64 - ) - err := rows.Scan( - &tableUpload.ID, - &tableUpload.UploadID, - &tableUpload.TableName, - &tableUpload.Status, - &tableUpload.Error, - &lastExecTimeRaw, - &totalEvents, - &tableUpload.CreatedAt, - &tableUpload.UpdatedAt, - &locationRaw, - ) - if err != nil { - return nil, fmt.Errorf("scanning row: %w", err) - } - - tableUpload.CreatedAt = tableUpload.CreatedAt.UTC() - tableUpload.UpdatedAt = tableUpload.UpdatedAt.UTC() +func scanTableUpload(scan scanFn, tableUpload *model.TableUpload) error { + var ( + locationRaw sql.NullString + lastExecTimeRaw sql.NullTime + totalEvents sql.NullInt64 + ) + err := scan( + &tableUpload.ID, + &tableUpload.UploadID, + &tableUpload.TableName, + &tableUpload.Status, + &tableUpload.Error, + &lastExecTimeRaw, + &totalEvents, + &tableUpload.CreatedAt, + &tableUpload.UpdatedAt, + &locationRaw, + ) + if err != nil { + return fmt.Errorf("scanning row: %w", err) + } - if lastExecTimeRaw.Valid { - tableUpload.LastExecTime = lastExecTimeRaw.Time.UTC() - } - if locationRaw.Valid { - tableUpload.Location = locationRaw.String - } - if totalEvents.Valid { - tableUpload.TotalEvents = totalEvents.Int64 - } + tableUpload.CreatedAt = tableUpload.CreatedAt.UTC() + tableUpload.UpdatedAt = tableUpload.UpdatedAt.UTC() - tableUploads = append(tableUploads, tableUpload) + if lastExecTimeRaw.Valid { + tableUpload.LastExecTime = lastExecTimeRaw.Time.UTC() } - - if err := rows.Err(); err != nil { - return nil, fmt.Errorf("iterating rows: %w", err) + if locationRaw.Valid { + tableUpload.Location = locationRaw.String } - - return tableUploads, nil + if totalEvents.Valid { + tableUpload.TotalEvents = totalEvents.Int64 + } + return nil } func (repo *TableUploads) PopulateTotalEventsFromStagingFileIDs(ctx context.Context, uploadId int64, tableName string, stagingFileIDs []int64) error { @@ -401,3 +406,54 @@ func (repo *TableUploads) SyncsInfo(ctx context.Context, uploadID int64) ([]mode }) return tableUploadInfos, nil } + +func (repo *TableUploads) GetByJobRunTaskRun( + ctx context.Context, + sourceID, + destinationID, + jobRunID, + taskRunID string, +) ([]model.TableUpload, error) { + rows, err := repo.db.QueryContext(ctx, ` + SELECT + `+tableUploadColumns+` + FROM + `+tableUploadTableName+` + WHERE + wh_upload_id IN ( + SELECT + id + FROM + `+uploadsTableName+` + WHERE + source_id=$1 AND + destination_id=$2 AND + metadata->>'source_job_run_id'=$3 AND + metadata->>'source_task_run_id'=$4 + ); + `, + sourceID, + destinationID, + jobRunID, + taskRunID, + ) + if err != nil { + return nil, fmt.Errorf("getting table uploads: %w", err) + } + defer func() { _ = rows.Close() }() + + var tableUploads []model.TableUpload + for rows.Next() { + var tableUpload model.TableUpload + err := scanTableUpload(rows.Scan, &tableUpload) + if err != nil { + return nil, err + } + tableUploads = append(tableUploads, tableUpload) + } + if err := rows.Err(); err != nil { + return nil, err + } + + return tableUploads, nil +} diff --git a/warehouse/jobs/http.go b/warehouse/jobs/http.go deleted file mode 100644 index 323d3568ca..0000000000 --- a/warehouse/jobs/http.go +++ /dev/null @@ -1,157 +0,0 @@ -package jobs - -import ( - "encoding/json" - "errors" - "fmt" - "net/http" - "strings" - - "github.com/rudderlabs/rudder-server/services/notifier" - - ierrors "github.com/rudderlabs/rudder-server/warehouse/internal/errors" - lf "github.com/rudderlabs/rudder-server/warehouse/logfield" - - "github.com/samber/lo" -) - -type insertJobResponse struct { - JobIds []int64 `json:"jobids"` - Err error `json:"error"` -} - -// InsertJobHandler adds a job to the warehouse_jobs table -func (a *AsyncJobWh) InsertJobHandler(w http.ResponseWriter, r *http.Request) { - defer func() { _ = r.Body.Close() }() - - if !a.enabled { - a.logger.Errorw("jobs api not initialized for inserting async job") - http.Error(w, ierrors.ErrJobsApiNotInitialized.Error(), http.StatusInternalServerError) - return - } - - var payload StartJobReqPayload - if err := json.NewDecoder(r.Body).Decode(&payload); err != nil { - a.logger.Warnw("invalid JSON in request body for inserting async jobs", lf.Error, err.Error()) - http.Error(w, ierrors.ErrInvalidJSONRequestBody.Error(), http.StatusBadRequest) - return - } - - if err := validatePayload(&payload); err != nil { - a.logger.Warnw("invalid payload for inserting async job", lf.Error, err.Error()) - http.Error(w, fmt.Sprintf("invalid payload: %s", err.Error()), http.StatusBadRequest) - return - } - - // TODO: Move to repository - tableNames, err := a.tableNamesBy(payload.SourceID, payload.DestinationID, payload.JobRunID, payload.TaskRunID) - if err != nil { - a.logger.Errorw("extracting tableNames for inserting async job", lf.Error, err.Error()) - http.Error(w, "can't extract tableNames", http.StatusInternalServerError) - return - } - - tableNames = lo.Filter(tableNames, func(tableName string, i int) bool { - switch strings.ToLower(tableName) { - case "rudder_discards", "rudder_identity_mappings", "rudder_identity_merge_rules": - return false - default: - return true - } - }) - - jobIds := make([]int64, 0, len(tableNames)) - for _, table := range tableNames { - metadataJson, err := json.Marshal(WhJobsMetaData{ - JobRunID: payload.JobRunID, - TaskRunID: payload.TaskRunID, - StartTime: payload.StartTime, - JobType: string(notifier.JobTypeAsync), - }) - if err != nil { - a.logger.Errorw("marshalling metadata for inserting async job", lf.Error, err.Error()) - http.Error(w, "can't marshall metadata", http.StatusInternalServerError) - return - } - - // TODO: Move to repository - id, err := a.addJobsToDB(&AsyncJobPayload{ - SourceID: payload.SourceID, - DestinationID: payload.DestinationID, - TableName: table, - AsyncJobType: payload.AsyncJobType, - MetaData: metadataJson, - WorkspaceID: payload.WorkspaceID, - }) - if err != nil { - a.logger.Errorw("inserting async job", lf.Error, err.Error()) - http.Error(w, "can't insert async job", http.StatusInternalServerError) - return - } - - jobIds = append(jobIds, id) - } - - resBody, err := json.Marshal(insertJobResponse{ - JobIds: jobIds, - Err: nil, - }) - if err != nil { - a.logger.Errorw("marshalling response for inserting async job", lf.Error, err.Error()) - http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) - return - } - - _, _ = w.Write(resBody) -} - -// StatusJobHandler The following handler gets called for getting the status of the async job -func (a *AsyncJobWh) StatusJobHandler(w http.ResponseWriter, r *http.Request) { - defer func() { _ = r.Body.Close() }() - - if !a.enabled { - a.logger.Errorw("jobs api not initialized for async job status") - http.Error(w, ierrors.ErrJobsApiNotInitialized.Error(), http.StatusInternalServerError) - return - } - - queryParams := r.URL.Query() - payload := StartJobReqPayload{ - TaskRunID: queryParams.Get("task_run_id"), - JobRunID: queryParams.Get("job_run_id"), - SourceID: queryParams.Get("source_id"), - DestinationID: queryParams.Get("destination_id"), - WorkspaceID: queryParams.Get("workspace_id"), - } - if err := validatePayload(&payload); err != nil { - a.logger.Warnw("invalid payload for async job status", lf.Error, err.Error()) - http.Error(w, fmt.Sprintf("invalid request: %s", err.Error()), http.StatusBadRequest) - return - } - - // TODO: Move to repository - jobStatus := a.jobStatus(&payload) - resBody, err := json.Marshal(jobStatus) - if err != nil { - a.logger.Errorw("marshalling response for async job status", lf.Error, err.Error()) - http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) - return - } - - _, _ = w.Write(resBody) -} - -func validatePayload(payload *StartJobReqPayload) error { - switch true { - case payload.SourceID == "": - return errors.New("source_id is required") - case payload.DestinationID == "": - return errors.New("destination_id is required") - case payload.JobRunID == "": - return errors.New("job_run_id is required") - case payload.TaskRunID == "": - return errors.New("task_run_id is required") - default: - return nil - } -} diff --git a/warehouse/jobs/jobs.go b/warehouse/jobs/jobs.go deleted file mode 100644 index eada87ea25..0000000000 --- a/warehouse/jobs/jobs.go +++ /dev/null @@ -1,64 +0,0 @@ -package jobs - -import ( - "context" - "time" - - "github.com/rudderlabs/rudder-server/warehouse/internal/model" - - warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" -) - -type WhAsyncJob struct{} - -func (*WhAsyncJob) IsWarehouseSchemaEmpty() bool { return true } - -func (*WhAsyncJob) GetLocalSchema(context.Context) (model.Schema, error) { - return model.Schema{}, nil -} - -func (*WhAsyncJob) UpdateLocalSchema(context.Context, model.Schema) error { - return nil -} - -func (*WhAsyncJob) GetTableSchemaInWarehouse(string) model.TableSchema { - return model.TableSchema{} -} - -func (*WhAsyncJob) GetTableSchemaInUpload(string) model.TableSchema { - return model.TableSchema{} -} - -func (*WhAsyncJob) GetLoadFilesMetadata(context.Context, warehouseutils.GetLoadFilesOptions) []warehouseutils.LoadFile { - return []warehouseutils.LoadFile{} -} - -func (*WhAsyncJob) GetSampleLoadFileLocation(context.Context, string) (string, error) { - return "", nil -} - -func (*WhAsyncJob) GetSingleLoadFile(context.Context, string) (warehouseutils.LoadFile, error) { - return warehouseutils.LoadFile{}, nil -} - -func (*WhAsyncJob) ShouldOnDedupUseNewRecord() bool { - return false -} - -func (*WhAsyncJob) UseRudderStorage() bool { - return false -} - -func (*WhAsyncJob) GetLoadFileGenStartTIme() time.Time { - return time.Time{} -} - -func (*WhAsyncJob) GetLoadFileType() string { - return "" -} - -func (*WhAsyncJob) GetFirstLastEvent() (time.Time, time.Time) { - return time.Now(), time.Now() -} - -func (*WhAsyncJob) CanAppend() bool { return false } diff --git a/warehouse/jobs/runner.go b/warehouse/jobs/runner.go deleted file mode 100644 index 3ae446c229..0000000000 --- a/warehouse/jobs/runner.go +++ /dev/null @@ -1,425 +0,0 @@ -package jobs - -import ( - "context" - "database/sql" - "encoding/json" - "fmt" - "time" - - sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" - - "github.com/rudderlabs/rudder-server/services/notifier" - - "github.com/lib/pq" - "github.com/samber/lo" - "golang.org/x/sync/errgroup" - - "github.com/rudderlabs/rudder-go-kit/config" - "github.com/rudderlabs/rudder-go-kit/logger" - "github.com/rudderlabs/rudder-server/utils/misc" - "github.com/rudderlabs/rudder-server/utils/timeutil" - warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" -) - -// New Initializes AsyncJobWh structure with appropriate variabless -func New( - ctx context.Context, - db *sqlmw.DB, - notifier *notifier.Notifier, -) *AsyncJobWh { - return &AsyncJobWh{ - db: db, - enabled: false, - notifier: notifier, - context: ctx, - logger: logger.NewLogger().Child("asyncjob"), - } -} - -func WithConfig(a *AsyncJobWh, config *config.Config) { - a.maxBatchSizeToProcess = config.GetInt("Warehouse.jobs.maxBatchSizeToProcess", 10) - a.maxCleanUpRetries = config.GetInt("Warehouse.jobs.maxCleanUpRetries", 5) - a.maxQueryRetries = config.GetInt("Warehouse.jobs.maxQueryRetries", 3) - a.maxAttemptsPerJob = config.GetInt("Warehouse.jobs.maxAttemptsPerJob", 3) - a.retryTimeInterval = config.GetDuration("Warehouse.jobs.retryTimeInterval", 10, time.Second) - a.asyncJobTimeOut = config.GetDuration("Warehouse.jobs.asyncJobTimeOut", 300, time.Second) -} - -func (a *AsyncJobWh) tableNamesBy(sourceID, destinationID, jobRunID, taskRunID string) ([]string, error) { - a.logger.Infof("[WH-Jobs]: Extracting table names for the job run id %s", jobRunID) - var tableNames []string - var err error - - query := `SELECT table_name FROM ` + warehouseutils.WarehouseTableUploadsTable + ` WHERE wh_upload_id IN ` + - ` (SELECT id FROM ` + warehouseutils.WarehouseUploadsTable + ` WHERE metadata->>'source_job_run_id'=$1 - AND metadata->>'source_task_run_id'=$2 - AND source_id=$3 - AND destination_id=$4)` - a.logger.Debugf("[WH-Jobs]: Query is %s\n", query) - rows, err := a.db.QueryContext(a.context, query, jobRunID, taskRunID, sourceID, destinationID) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error executing the query %s with error %v", query, err) - return nil, err - } - defer func() { _ = rows.Close() }() - for rows.Next() { - var tableName string - err = rows.Scan(&tableName) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error scanning the rows %s", err.Error()) - return nil, err - } - tableNames = append(tableNames, tableName) - } - if err = rows.Err(); err != nil { - a.logger.Errorf("[WH-Jobs]: Error iterating the rows %s", err.Error()) - return nil, err - } - a.logger.Infof("Got the TableNames as %s", tableNames) - return lo.Uniq(tableNames), nil -} - -// Takes AsyncJobPayload and adds rows to table wh_async_jobs -func (a *AsyncJobWh) addJobsToDB(payload *AsyncJobPayload) (int64, error) { - a.logger.Infof("[WH-Jobs]: Adding job to the wh_async_jobs %s for tableName: %s", payload.MetaData, payload.TableName) - var jobId int64 - sqlStatement := `INSERT INTO ` + warehouseutils.WarehouseAsyncJobTable + ` ( - source_id, destination_id, tablename, - status, created_at, updated_at, async_job_type, - workspace_id, metadata - ) - VALUES - ($1, $2, $3, $4, $5, $6 ,$7, $8, $9 ) RETURNING id` - - stmt, err := a.db.Prepare(sqlStatement) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error preparing out the query %s ", sqlStatement) - err = fmt.Errorf("error preparing out the query, while addJobsToDB %v", err) - return 0, err - } - - defer func() { _ = stmt.Close() }() - now := timeutil.Now() - row := stmt.QueryRowContext(a.context, payload.SourceID, payload.DestinationID, payload.TableName, WhJobWaiting, now, now, payload.AsyncJobType, payload.WorkspaceID, payload.MetaData) - err = row.Scan(&jobId) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error processing the %s, %s ", sqlStatement, err.Error()) - return 0, err - } - return jobId, nil -} - -// Run Async Job runner's main job is to -// 1. Scan the database for entries into wh_async_jobs -// 2. Publish data to pg_notifier queue -// 3. Move any executing jobs to waiting -func (a *AsyncJobWh) Run() error { - // Start the asyncJobRunner - a.logger.Info("[WH-Jobs]: Initializing async job runner") - g, ctx := errgroup.WithContext(a.context) - a.context = ctx - err := misc.RetryWith(a.context, a.retryTimeInterval, a.maxCleanUpRetries, func(ctx context.Context) error { - err := a.cleanUpAsyncTable(ctx) - if err != nil { - a.logger.Errorf("[WH-Jobs]: unable to cleanup asynctable with error %s", err.Error()) - return err - } - a.enabled = true - return nil - }) - if err != nil { - a.logger.Errorf("[WH-Jobs]: unable to cleanup asynctable with error %s", err.Error()) - return err - } - if a.enabled { - g.Go(func() error { - return a.startAsyncJobRunner(ctx) - }) - } - return g.Wait() -} - -func (a *AsyncJobWh) cleanUpAsyncTable(ctx context.Context) error { - a.logger.Info("[WH-Jobs]: Cleaning up the zombie asyncjobs") - sqlStatement := fmt.Sprintf( - `UPDATE %s SET status=$1 WHERE status=$2 or status=$3`, - pq.QuoteIdentifier(warehouseutils.WarehouseAsyncJobTable), - ) - a.logger.Debugf("[WH-Jobs]: resetting up async jobs table query %s", sqlStatement) - _, err := a.db.ExecContext(ctx, sqlStatement, WhJobWaiting, WhJobExecuting, WhJobFailed) - return err -} - -/* -startAsyncJobRunner is the main runner that -1) Periodically queries the db for any pending async jobs -2) Groups them together -3) Publishes them to the notifier -4) Spawns a subroutine that periodically checks for responses from Notifier/slave worker post trackBatch -*/ -func (a *AsyncJobWh) startAsyncJobRunner(ctx context.Context) error { - a.logger.Info("[WH-Jobs]: Starting async job runner") - defer a.logger.Info("[WH-Jobs]: Stopping AsyncJobRunner") - - for { - a.logger.Debug("[WH-Jobs]: Scanning for waiting async job") - - select { - case <-ctx.Done(): - return nil - case <-time.After(a.retryTimeInterval): - } - - pendingAsyncJobs, err := a.getPendingAsyncJobs(ctx) - if err != nil { - a.logger.Errorf("[WH-Jobs]: unable to get pending async jobs with error %s", err.Error()) - continue - } - if len(pendingAsyncJobs) == 0 { - continue - } - - a.logger.Infof("[WH-Jobs]: Number of async wh jobs left = %d", len(pendingAsyncJobs)) - - notifierClaims, err := getMessagePayloadsFromAsyncJobPayloads(pendingAsyncJobs) - if err != nil { - a.logger.Errorf("Error converting the asyncJobType to notifier payload %s ", err) - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobFailed, err) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - continue - } - ch, err := a.notifier.Publish(ctx, ¬ifier.PublishRequest{ - Payloads: notifierClaims, - JobType: notifier.JobTypeAsync, - Priority: 100, - }) - if err != nil { - a.logger.Errorf("[WH-Jobs]: unable to get publish async jobs to notifier. Task failed with error %s", err.Error()) - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobFailed, err) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - continue - } - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobExecuting, err) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - - select { - case <-ctx.Done(): - a.logger.Infof("[WH-Jobs]: Context cancelled for async job runner") - return nil - case responses, ok := <-ch: - if !ok { - a.logger.Error("[WH-Jobs]: Notifier track batch channel closed") - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobFailed, fmt.Errorf("receiving channel closed")) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - continue - } - if responses.Err != nil { - a.logger.Errorf("[WH-Jobs]: Error received from the notifier track batch %s", responses.Err.Error()) - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobFailed, responses.Err) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - continue - } - a.logger.Info("[WH-Jobs]: Response received from the notifier track batch") - asyncJobsStatusMap := getAsyncStatusMapFromAsyncPayloads(pendingAsyncJobs) - a.updateStatusJobPayloadsFromNotifierResponse(responses, asyncJobsStatusMap) - _ = a.updateAsyncJobs(ctx, asyncJobsStatusMap) - case <-time.After(a.asyncJobTimeOut): - a.logger.Errorf("Go Routine timed out waiting for a response from notifier", pendingAsyncJobs[0].Id) - asyncJobStatusMap := convertToPayloadStatusStructWithSingleStatus(pendingAsyncJobs, WhJobFailed, err) - _ = a.updateAsyncJobs(ctx, asyncJobStatusMap) - } - } -} - -func (a *AsyncJobWh) updateStatusJobPayloadsFromNotifierResponse(r *notifier.PublishResponse, m map[string]AsyncJobStatus) { - for _, resp := range r.Jobs { - var response NotifierResponse - err := json.Unmarshal(resp.Payload, &response) - if err != nil { - a.logger.Errorf("error unmarshalling notifier payload to AsyncJobStatusMa for Id: %s", response.Id) - continue - } - - if output, ok := m[response.Id]; ok { - output.Status = string(resp.Status) - if resp.Error != nil { - output.Error = fmt.Errorf(resp.Error.Error()) - } - m[response.Id] = output - } - } -} - -// Queries the jobsDB and gets active async job and returns it in a -func (a *AsyncJobWh) getPendingAsyncJobs(ctx context.Context) ([]AsyncJobPayload, error) { - asyncJobPayloads := make([]AsyncJobPayload, 0) - a.logger.Debug("[WH-Jobs]: Get pending wh async jobs") - // Filter to get most recent row for the sourceId/destinationID combo and remaining ones should relegate to abort. - var attempt int - query := fmt.Sprintf( - `SELECT - id, - source_id, - destination_id, - tablename, - async_job_type, - metadata, - attempt - FROM %s WHERE (status=$1 OR status=$2) LIMIT $3`, warehouseutils.WarehouseAsyncJobTable) - rows, err := a.db.QueryContext(ctx, query, WhJobWaiting, WhJobFailed, a.maxBatchSizeToProcess) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error in getting pending wh async jobs with error %s", err.Error()) - return asyncJobPayloads, err - } - defer func() { _ = rows.Close() }() - for rows.Next() { - var asyncJobPayload AsyncJobPayload - err = rows.Scan( - &asyncJobPayload.Id, - &asyncJobPayload.SourceID, - &asyncJobPayload.DestinationID, - &asyncJobPayload.TableName, - &asyncJobPayload.AsyncJobType, - &asyncJobPayload.MetaData, - &attempt, - ) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error scanning rows %s\n", err) - return asyncJobPayloads, err - } - asyncJobPayloads = append(asyncJobPayloads, asyncJobPayload) - a.logger.Infof("Adding row with Id = %s & attempt no %d", asyncJobPayload.Id, attempt) - } - if err := rows.Err(); err != nil { - a.logger.Errorf("[WH-Jobs]: Error in getting pending wh async jobs with error %s", rows.Err().Error()) - return asyncJobPayloads, err - } - return asyncJobPayloads, nil -} - -// Updates the warehouse async jobs with the status sent as a parameter -func (a *AsyncJobWh) updateAsyncJobs(ctx context.Context, payloads map[string]AsyncJobStatus) error { - a.logger.Info("[WH-Jobs]: Updating wh async jobs to Executing") - var err error - for _, payload := range payloads { - if payload.Error != nil { - err = a.updateAsyncJobStatus(ctx, payload.Id, payload.Status, payload.Error.Error()) - continue - } - err = a.updateAsyncJobStatus(ctx, payload.Id, payload.Status, "") - } - return err -} - -func (a *AsyncJobWh) updateAsyncJobStatus(ctx context.Context, Id, status, errMessage string) error { - a.logger.Infof("[WH-Jobs]: Updating status of wh async jobs to %s", status) - sqlStatement := fmt.Sprintf(`UPDATE %s SET status=(CASE - WHEN attempt >= $1 - THEN $2 - ELSE $3 - END) , - error=$4 WHERE id=$5 AND status!=$6 AND status!=$7 `, - warehouseutils.WarehouseAsyncJobTable, - ) - var err error - for retryCount := 0; retryCount < a.maxQueryRetries; retryCount++ { - a.logger.Debugf("[WH-Jobs]: updating async jobs table query %s, retry no : %d", sqlStatement, retryCount) - _, err := a.db.ExecContext(ctx, sqlStatement, - a.maxAttemptsPerJob, WhJobAborted, status, errMessage, Id, WhJobAborted, WhJobSucceeded, - ) - if err == nil { - a.logger.Info("Update successful") - a.logger.Debugf("query: %s successfully executed", sqlStatement) - if status == WhJobFailed { - return a.updateAsyncJobAttempt(ctx, Id) - } - return err - } - } - - a.logger.Errorf("Query: %s failed with error: %s", sqlStatement, err.Error()) - return err -} - -func (a *AsyncJobWh) updateAsyncJobAttempt(ctx context.Context, Id string) error { - a.logger.Info("[WH-Jobs]: Incrementing wh async jobs attempt") - sqlStatement := fmt.Sprintf(`UPDATE %s SET attempt=attempt+1 WHERE id=$1 AND status!=$2 AND status!=$3 `, warehouseutils.WarehouseAsyncJobTable) - var err error - for queryRetry := 0; queryRetry < a.maxQueryRetries; queryRetry++ { - a.logger.Debugf("[WH-Jobs]: updating async jobs table query %s, retry no : %d", sqlStatement, queryRetry) - row, err := a.db.QueryContext(ctx, sqlStatement, Id, WhJobAborted, WhJobSucceeded) - if err == nil { - a.logger.Info("Update successful") - a.logger.Debugf("query: %s successfully executed", sqlStatement) - return nil - } - _ = row.Err() - } - a.logger.Errorf("query: %s failed with Error : %s", sqlStatement, err.Error()) - return err -} - -// returns status and errMessage -// Only succeeded, executing & waiting states should have empty errMessage -// Rest of the states failed, aborted should send an error message conveying a message -func (a *AsyncJobWh) jobStatus(payload *StartJobReqPayload) WhStatusResponse { - var statusResponse WhStatusResponse - a.logger.Info("[WH-Jobs]: Getting status for wh async jobs %v", payload) - // Need to check for count first and see if there are any rows matching the job_run_id and task_run_id. If none, then raise an error instead of showing complete - sqlStatement := fmt.Sprintf(`SELECT status,error FROM %s WHERE metadata->>'job_run_id'=$1 AND metadata->>'task_run_id'=$2`, warehouseutils.WarehouseAsyncJobTable) - a.logger.Debugf("Query inside getStatusAsync function is %s", sqlStatement) - rows, err := a.db.QueryContext(a.context, sqlStatement, payload.JobRunID, payload.TaskRunID) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error executing the query %s", err.Error()) - return WhStatusResponse{ - Status: WhJobFailed, - Err: err.Error(), - } - } - defer func() { _ = rows.Close() }() - for rows.Next() { - var status string - var errMessage sql.NullString - err = rows.Scan(&status, &errMessage) - if err != nil { - a.logger.Errorf("[WH-Jobs]: Error scanning rows %s\n", err) - return WhStatusResponse{ - Status: WhJobFailed, - Err: err.Error(), - } - } - - switch status { - case WhJobFailed: - a.logger.Infof("[WH-Jobs] Async Job with job_run_id: %s, task_run_id: %s is failed", payload.JobRunID, payload.TaskRunID) - statusResponse.Status = WhJobFailed - if !errMessage.Valid { - statusResponse.Err = "Failed while scanning" - } - statusResponse.Err = errMessage.String - case WhJobAborted: - a.logger.Infof("[WH-Jobs] Async Job with job_run_id: %s, task_run_id: %s is aborted", payload.JobRunID, payload.TaskRunID) - statusResponse.Status = WhJobAborted - if !errMessage.Valid { - statusResponse.Err = "Failed while scanning" - } - statusResponse.Err = errMessage.String - case WhJobSucceeded: - a.logger.Infof("[WH-Jobs] Async Job with job_run_id: %s, task_run_id: %s is complete", payload.JobRunID, payload.TaskRunID) - statusResponse.Status = WhJobSucceeded - default: - a.logger.Infof("[WH-Jobs] Async Job with job_run_id: %s, task_run_id: %s is under processing", payload.JobRunID, payload.TaskRunID) - statusResponse.Status = WhJobExecuting - } - } - - if err = rows.Err(); err != nil { - a.logger.Errorf("[WH-Jobs]: Error scanning rows %s\n", err) - return WhStatusResponse{ - Status: WhJobFailed, - Err: err.Error(), - } - } - return statusResponse -} diff --git a/warehouse/jobs/types.go b/warehouse/jobs/types.go deleted file mode 100644 index f85fafab49..0000000000 --- a/warehouse/jobs/types.go +++ /dev/null @@ -1,89 +0,0 @@ -package jobs - -import ( - "context" - "encoding/json" - "time" - - "github.com/rudderlabs/rudder-server/services/notifier" - - sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" - - "github.com/rudderlabs/rudder-go-kit/logger" -) - -// StartJobReqPayload For processing requests payload in handlers.go -type StartJobReqPayload struct { - SourceID string `json:"source_id"` - Type string `json:"type"` - Channel string `json:"channel"` - DestinationID string `json:"destination_id"` - StartTime string `json:"start_time"` - JobRunID string `json:"job_run_id"` - TaskRunID string `json:"task_run_id"` - AsyncJobType string `json:"async_job_type"` - WorkspaceID string `json:"workspace_id"` -} - -type AsyncJobWh struct { - db *sqlmw.DB - enabled bool - notifier *notifier.Notifier - context context.Context - logger logger.Logger - maxBatchSizeToProcess int - maxCleanUpRetries int - maxQueryRetries int - retryTimeInterval time.Duration - maxAttemptsPerJob int - asyncJobTimeOut time.Duration -} - -type WhJobsMetaData struct { - JobRunID string `json:"job_run_id"` - TaskRunID string `json:"task_run_id"` - JobType string `json:"jobtype"` - StartTime string `json:"start_time"` -} - -// AsyncJobPayload For creating job payload to wh_async_jobs table -type AsyncJobPayload struct { - Id string `json:"id"` - SourceID string `json:"source_id"` - DestinationID string `json:"destination_id"` - TableName string `json:"tablename"` - AsyncJobType string `json:"async_job_type"` - WorkspaceID string `json:"workspace_id"` - MetaData json.RawMessage `json:"metadata"` -} - -const ( - WhJobWaiting string = "waiting" - WhJobExecuting string = "executing" - WhJobSucceeded string = "succeeded" - WhJobAborted string = "aborted" - WhJobFailed string = "failed" -) - -type NotifierResponse struct { - Id string `json:"id"` -} - -type WhStatusResponse struct { - Status string - Err string -} - -type WhAsyncJobRunner interface { - startAsyncJobRunner(context.Context) - getTableNamesBy(context.Context, string, string) - getPendingAsyncJobs(context.Context) ([]AsyncJobPayload, error) - getStatusAsyncJob(*StartJobReqPayload) (string, error) - updateMultipleAsyncJobs(*[]AsyncJobPayload, string, string) -} - -type AsyncJobStatus struct { - Id string - Status string - Error error -} diff --git a/warehouse/jobs/utils.go b/warehouse/jobs/utils.go deleted file mode 100644 index 017797a8ef..0000000000 --- a/warehouse/jobs/utils.go +++ /dev/null @@ -1,41 +0,0 @@ -package jobs - -import ( - "encoding/json" -) - -func convertToPayloadStatusStructWithSingleStatus(payloads []AsyncJobPayload, status string, err error) map[string]AsyncJobStatus { - asyncJobStatusMap := make(map[string]AsyncJobStatus) - for _, payload := range payloads { - asyncJobStatusMap[payload.Id] = AsyncJobStatus{ - Id: payload.Id, - Status: status, - Error: err, - } - } - return asyncJobStatusMap -} - -// convert to notifier Payload and return the array of payloads -func getMessagePayloadsFromAsyncJobPayloads(asyncJobPayloads []AsyncJobPayload) ([]json.RawMessage, error) { - var messages []json.RawMessage - for _, job := range asyncJobPayloads { - message, err := json.Marshal(job) - if err != nil { - return messages, err - } - messages = append(messages, message) - } - return messages, nil -} - -func getAsyncStatusMapFromAsyncPayloads(payloads []AsyncJobPayload) map[string]AsyncJobStatus { - asyncJobStatusMap := make(map[string]AsyncJobStatus) - for _, payload := range payloads { - asyncJobStatusMap[payload.Id] = AsyncJobStatus{ - Id: payload.Id, - Status: WhJobFailed, - } - } - return asyncJobStatusMap -} diff --git a/warehouse/slave_worker.go b/warehouse/slave_worker.go index f8c54423e2..5dcc2ba245 100644 --- a/warehouse/slave_worker.go +++ b/warehouse/slave_worker.go @@ -23,7 +23,7 @@ import ( integrationsconfig "github.com/rudderlabs/rudder-server/warehouse/integrations/config" "github.com/rudderlabs/rudder-server/warehouse/integrations/manager" "github.com/rudderlabs/rudder-server/warehouse/internal/model" - "github.com/rudderlabs/rudder-server/warehouse/jobs" + "github.com/rudderlabs/rudder-server/warehouse/source" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" ) @@ -48,8 +48,8 @@ type uploadResult struct { } type asyncJobRunResult struct { - Result bool `json:"Result"` - ID string `json:"Id"` + Result bool `json:"Result"` + ID int64 `json:"Id"` } type slaveWorker struct { @@ -430,7 +430,7 @@ func (sw *slaveWorker) processClaimedAsyncJob(ctx context.Context, claimedJob *n } var ( - job jobs.AsyncJobPayload + job model.SourceJob err error ) @@ -456,9 +456,9 @@ func (sw *slaveWorker) processClaimedAsyncJob(ctx context.Context, claimedJob *n }) } -func (sw *slaveWorker) runAsyncJob(ctx context.Context, asyncjob jobs.AsyncJobPayload) (asyncJobRunResult, error) { +func (sw *slaveWorker) runAsyncJob(ctx context.Context, asyncjob model.SourceJob) (asyncJobRunResult, error) { result := asyncJobRunResult{ - ID: asyncjob.Id, + ID: asyncjob.ID, Result: false, } @@ -477,19 +477,19 @@ func (sw *slaveWorker) runAsyncJob(ctx context.Context, asyncjob jobs.AsyncJobPa warehouse.Destination.ID, )) - err = integrationsManager.Setup(ctx, warehouse, &jobs.WhAsyncJob{}) + err = integrationsManager.Setup(ctx, warehouse, &source.SourceUploader{}) if err != nil { return result, err } defer integrationsManager.Cleanup(ctx) var metadata warehouseutils.DeleteByMetaData - if err = json.Unmarshal(asyncjob.MetaData, &metadata); err != nil { + if err = json.Unmarshal(asyncjob.Metadata, &metadata); err != nil { return result, err } - switch asyncjob.AsyncJobType { - case "deletebyjobrunid": + switch asyncjob.JobType { + case model.DeleteByJobRunID: err = integrationsManager.DeleteBy(ctx, []string{asyncjob.TableName}, warehouseutils.DeleteByParams{ SourceId: asyncjob.SourceID, TaskRunId: metadata.TaskRunId, diff --git a/warehouse/slave_worker_test.go b/warehouse/slave_worker_test.go index 42a1803e51..5e0ffde2fc 100644 --- a/warehouse/slave_worker_test.go +++ b/warehouse/slave_worker_test.go @@ -30,7 +30,6 @@ import ( "github.com/rudderlabs/rudder-server/utils/misc" "github.com/rudderlabs/rudder-server/utils/pubsub" "github.com/rudderlabs/rudder-server/warehouse/internal/model" - "github.com/rudderlabs/rudder-server/warehouse/jobs" "github.com/rudderlabs/rudder-server/warehouse/multitenant" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" ) @@ -574,14 +573,14 @@ func TestSlaveWorker(t *testing.T) { workerIdx, ) - p := jobs.AsyncJobPayload{ - Id: "1", + p := model.SourceJob{ + ID: 1, SourceID: sourceID, DestinationID: destinationID, TableName: "test_table_name", WorkspaceID: workspaceID, - AsyncJobType: "deletebyjobrunid", - MetaData: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), + JobType: model.DeleteByJobRunID, + Metadata: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), } payloadJson, err := json.Marshal(p) @@ -656,21 +655,21 @@ func TestSlaveWorker(t *testing.T) { }, { name: "invalid parameters", - jobType: "deletebyjobrunid", + jobType: model.DeleteByJobRunID, expectedError: errors.New("invalid Parameters"), }, { name: "invalid source id", sourceID: "invalid_source_id", destinationID: destinationID, - jobType: "deletebyjobrunid", + jobType: model.DeleteByJobRunID, expectedError: errors.New("invalid Source Id"), }, { name: "invalid destination id", sourceID: sourceID, destinationID: "invalid_destination_id", - jobType: "deletebyjobrunid", + jobType: model.DeleteByJobRunID, expectedError: errors.New("invalid Destination Id"), }, } @@ -679,14 +678,14 @@ func TestSlaveWorker(t *testing.T) { tc := tc t.Run(tc.name, func(t *testing.T) { - p := jobs.AsyncJobPayload{ - Id: "1", + p := model.SourceJob{ + ID: 1, SourceID: tc.sourceID, DestinationID: tc.destinationID, TableName: "test_table_name", WorkspaceID: workspaceID, - AsyncJobType: tc.jobType, - MetaData: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), + JobType: tc.jobType, + Metadata: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), } payloadJson, err := json.Marshal(p) diff --git a/warehouse/source/http.go b/warehouse/source/http.go new file mode 100644 index 0000000000..cd91adb2ab --- /dev/null +++ b/warehouse/source/http.go @@ -0,0 +1,208 @@ +package source + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "github.com/rudderlabs/rudder-server/warehouse/internal/model" + "net/http" + "strings" + + "github.com/rudderlabs/rudder-server/services/notifier" + + ierrors "github.com/rudderlabs/rudder-server/warehouse/internal/errors" + lf "github.com/rudderlabs/rudder-server/warehouse/logfield" + + "github.com/samber/lo" +) + +type insertJobRequest struct { + SourceID string `json:"source_id"` + Type string `json:"type"` + Channel string `json:"channel"` + DestinationID string `json:"destination_id"` + StartTime string `json:"start_time"` + JobRunID string `json:"job_run_id"` + TaskRunID string `json:"task_run_id"` + AsyncJobType string `json:"async_job_type"` + WorkspaceID string `json:"workspace_id"` +} + +type insertJobResponse struct { + JobIds []int64 `json:"jobids"` + Err error `json:"error"` +} + +type jobStatusResponse struct { + Status string + Err string +} + +// InsertJobHandler adds a job to the warehouse_jobs table +func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { + defer func() { _ = r.Body.Close() }() + + var payload insertJobRequest + if err := json.NewDecoder(r.Body).Decode(&payload); err != nil { + a.logger.Warnw("invalid JSON in request body for inserting async jobs", lf.Error, err.Error()) + http.Error(w, ierrors.ErrInvalidJSONRequestBody.Error(), http.StatusBadRequest) + return + } + + if err := validatePayload(&payload); err != nil { + a.logger.Warnw("invalid payload for inserting async job", lf.Error, err.Error()) + http.Error(w, fmt.Sprintf("invalid payload: %s", err.Error()), http.StatusBadRequest) + return + } + + tableUploads, err := a.tableUploadsRepo.GetByJobRunTaskRun( + r.Context(), + payload.SourceID, + payload.DestinationID, + payload.JobRunID, + payload.TaskRunID, + ) + if err != nil { + if errors.Is(r.Context().Err(), context.Canceled) { + http.Error(w, ierrors.ErrRequestCancelled.Error(), http.StatusBadRequest) + return + } + a.logger.Errorw("extracting tableNames for inserting async job", lf.Error, err.Error()) + http.Error(w, "can't extract tableNames", http.StatusInternalServerError) + return + } + + tableNames := lo.Map(tableUploads, func(item model.TableUpload, index int) string { + return item.TableName + }) + tableNames = lo.Filter(tableNames, func(tableName string, i int) bool { + switch strings.ToLower(tableName) { + case "rudder_discards", "rudder_identity_mappings", "rudder_identity_merge_rules": + return false + default: + return true + } + }) + + type metadata struct { + JobRunID string `json:"job_run_id"` + TaskRunID string `json:"task_run_id"` + JobType string `json:"jobtype"` + StartTime string `json:"start_time"` + } + + metadataJson, err := json.Marshal(metadata{ + JobRunID: payload.JobRunID, + TaskRunID: payload.TaskRunID, + StartTime: payload.StartTime, + JobType: string(notifier.JobTypeAsync), + }) + if err != nil { + a.logger.Errorw("marshalling metadata for inserting async job", lf.Error, err.Error()) + http.Error(w, "can't marshall metadata", http.StatusInternalServerError) + return + } + + sourceJobs := lo.Map(tableNames, func(item string, index int) model.SourceJob { + return model.SourceJob{ + SourceID: payload.SourceID, + DestinationID: payload.DestinationID, + WorkspaceID: payload.WorkspaceID, + TableName: item, + JobType: model.DeleteByJobRunID, + Metadata: metadataJson, + } + }) + jobIds, err := a.sourceRepo.Insert(r.Context(), sourceJobs) + if err != nil { + if errors.Is(r.Context().Err(), context.Canceled) { + http.Error(w, ierrors.ErrRequestCancelled.Error(), http.StatusBadRequest) + return + } + a.logger.Errorw("inserting source jobs", lf.Error, err.Error()) + http.Error(w, "can't insert source jobs", http.StatusInternalServerError) + return + } + + resBody, err := json.Marshal(insertJobResponse{ + JobIds: jobIds, + Err: nil, + }) + if err != nil { + a.logger.Errorw("marshalling response for inserting async job", lf.Error, err.Error()) + http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) + return + } + + _, _ = w.Write(resBody) +} + +// StatusJobHandler The following handler gets called for getting the status of the async job +func (a *Manager) StatusJobHandler(w http.ResponseWriter, r *http.Request) { + defer func() { _ = r.Body.Close() }() + + queryParams := r.URL.Query() + payload := insertJobRequest{ + TaskRunID: queryParams.Get("task_run_id"), + JobRunID: queryParams.Get("job_run_id"), + SourceID: queryParams.Get("source_id"), + DestinationID: queryParams.Get("destination_id"), + WorkspaceID: queryParams.Get("workspace_id"), + } + if err := validatePayload(&payload); err != nil { + a.logger.Warnw("invalid payload for async job status", lf.Error, err.Error()) + http.Error(w, fmt.Sprintf("invalid request: %s", err.Error()), http.StatusBadRequest) + return + } + + sourceJob, err := a.sourceRepo.GetByJobRunTaskRun(r.Context(), payload.JobRunID, payload.TaskRunID) + if err != nil { + if errors.Is(r.Context().Err(), context.Canceled) { + http.Error(w, ierrors.ErrRequestCancelled.Error(), http.StatusBadRequest) + return + } + a.logger.Warnw("unable to get async job status", lf.Error, err.Error()) + http.Error(w, fmt.Sprintf("can't get async job status: %s", err.Error()), http.StatusBadRequest) + return + } + + var statusResponse jobStatusResponse + switch sourceJob.Status { + case model.SourceJobStatusFailed, model.SourceJobStatusAborted: + errorMessage := "source job failed" + if sourceJob.Error != nil { + errorMessage = sourceJob.Error.Error() + } + statusResponse.Status = sourceJob.Status + statusResponse.Err = errorMessage + case model.SourceJobStatusSucceeded: + statusResponse.Status = sourceJob.Status + default: + statusResponse.Status = model.SourceJobStatusExecuting + } + + resBody, err := json.Marshal(statusResponse) + if err != nil { + a.logger.Errorw("marshalling response for async job status", lf.Error, err.Error()) + http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) + return + } + + _, _ = w.Write(resBody) +} + +func validatePayload(payload *insertJobRequest) error { + switch true { + case payload.SourceID == "": + return errors.New("source_id is required") + case payload.DestinationID == "": + return errors.New("destination_id is required") + case payload.JobRunID == "": + return errors.New("job_run_id is required") + case payload.TaskRunID == "": + return errors.New("task_run_id is required") + default: + return nil + } +} diff --git a/warehouse/jobs/http_test.go b/warehouse/source/http_test.go similarity index 95% rename from warehouse/jobs/http_test.go rename to warehouse/source/http_test.go index 47b8c7d1f2..243401a6d5 100644 --- a/warehouse/jobs/http_test.go +++ b/warehouse/source/http_test.go @@ -1,4 +1,4 @@ -package jobs +package source import ( "bytes" @@ -131,12 +131,12 @@ func TestAsyncJobHandlers(t *testing.T) { t.Run("validate payload", func(t *testing.T) { testCases := []struct { name string - payload StartJobReqPayload + payload insertJobRequest expectedError error }{ { name: "invalid source", - payload: StartJobReqPayload{ + payload: insertJobRequest{ JobRunID: "job_run_id", TaskRunID: "task_run_id", SourceID: "", @@ -147,7 +147,7 @@ func TestAsyncJobHandlers(t *testing.T) { }, { name: "invalid destination", - payload: StartJobReqPayload{ + payload: insertJobRequest{ JobRunID: "job_run_id", TaskRunID: "task_run_id", SourceID: "source_id", @@ -158,7 +158,7 @@ func TestAsyncJobHandlers(t *testing.T) { }, { name: "invalid task run", - payload: StartJobReqPayload{ + payload: insertJobRequest{ JobRunID: "job_run_id", TaskRunID: "", SourceID: "source_id", @@ -169,7 +169,7 @@ func TestAsyncJobHandlers(t *testing.T) { }, { name: "invalid job run", - payload: StartJobReqPayload{ + payload: insertJobRequest{ JobRunID: "", TaskRunID: "task_run_id", SourceID: "source_id", @@ -180,7 +180,7 @@ func TestAsyncJobHandlers(t *testing.T) { }, { name: "valid payload", - payload: StartJobReqPayload{ + payload: insertJobRequest{ JobRunID: "job_run_id", TaskRunID: "task_run_id", SourceID: "source_id", @@ -203,7 +203,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", nil) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: false, logger: logger.NOP, @@ -221,7 +221,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", bytes.NewReader([]byte(`"Invalid payload"`))) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: true, logger: logger.NOP, @@ -239,7 +239,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", bytes.NewReader([]byte(`{}`))) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: true, logger: logger.NOP, @@ -264,7 +264,7 @@ func TestAsyncJobHandlers(t *testing.T) { `))) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: true, logger: logger.NOP, @@ -287,7 +287,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: false, logger: logger.NOP, @@ -305,7 +305,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: true, logger: logger.NOP, @@ -336,7 +336,7 @@ func TestAsyncJobHandlers(t *testing.T) { req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status?"+qp.Encode(), nil) resp := httptest.NewRecorder() - jobsManager := AsyncJobWh{ + jobsManager := Manager{ db: db, enabled: true, logger: logger.NOP, @@ -346,7 +346,7 @@ func TestAsyncJobHandlers(t *testing.T) { jobsManager.StatusJobHandler(resp, req) require.Equal(t, http.StatusOK, resp.Code) - var statusResponse WhStatusResponse + var statusResponse JobStatusResponse err = json.NewDecoder(resp.Body).Decode(&statusResponse) require.NoError(t, err) require.Equal(t, statusResponse.Status, "aborted") diff --git a/warehouse/source/source.go b/warehouse/source/source.go new file mode 100644 index 0000000000..0c2cebef51 --- /dev/null +++ b/warehouse/source/source.go @@ -0,0 +1,224 @@ +package source + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "github.com/rudderlabs/rudder-server/warehouse/internal/model" + "github.com/rudderlabs/rudder-server/warehouse/internal/repo" + warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" + "time" + + sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" + + "github.com/rudderlabs/rudder-server/services/notifier" + + "github.com/rudderlabs/rudder-go-kit/config" + "github.com/rudderlabs/rudder-go-kit/logger" +) + +type notifierResponse struct { + Id int64 `json:"id"` +} + +type Manager struct { + logger logger.Logger + sourceRepo *repo.Source + tableUploadsRepo *repo.TableUploads + notifier *notifier.Notifier + + config struct { + maxBatchSizeToProcess int64 + processingSleepInterval time.Duration + maxAttemptsPerJob int + processingTimeout time.Duration + } +} + +// New Initializes AsyncJobWh structure with appropriate variabless +func New( + conf *config.Config, + log logger.Logger, + db *sqlmw.DB, + notifier *notifier.Notifier, +) *Manager { + m := &Manager{ + logger: log.Child("source-manager"), + tableUploadsRepo: repo.NewTableUploads(db), + sourceRepo: repo.NewSource(db), + notifier: notifier, + } + + m.config.maxBatchSizeToProcess = conf.GetInt64("Warehouse.jobs.maxBatchSizeToProcess", 10) + m.config.maxAttemptsPerJob = conf.GetInt("Warehouse.jobs.maxAttemptsPerJob", 3) + m.config.processingSleepInterval = conf.GetDuration("Warehouse.jobs.processingSleepInterval", 10, time.Second) + m.config.processingTimeout = conf.GetDuration("Warehouse.jobs.processingTimeout", 300, time.Second) + + return m +} + +func (a *Manager) Run(ctx context.Context) error { + if err := a.sourceRepo.Reset(ctx); err != nil { + return fmt.Errorf("unable to reset source table with error %s", err.Error()) + } + + return a.startProcessing(ctx) +} + +/* +startProcessing is the main runner that +1) Periodically queries the db for any pending async jobs +2) Groups them together +3) Publishes them to the notifier +4) Spawns a subroutine that periodically checks for responses from Notifier/slave worker post trackBatch +*/ +func (a *Manager) startProcessing(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return nil + case <-time.After(a.config.processingSleepInterval): + } + + pendingJobs, err := a.sourceRepo.GetToProcess(ctx, a.config.maxBatchSizeToProcess) + if err != nil { + return fmt.Errorf("unable to get pending async jobs with error %s", err.Error()) + } + if len(pendingJobs) == 0 { + continue + } + + notifierClaims := make([]json.RawMessage, 0, len(pendingJobs)) + for _, job := range pendingJobs { + message, err := json.Marshal(job) + if err != nil { + return fmt.Errorf("unable to marshal async job payload with error %s", err.Error()) + } + notifierClaims = append(notifierClaims, message) + } + + ch, err := a.notifier.Publish(ctx, ¬ifier.PublishRequest{ + Payloads: notifierClaims, + JobType: notifier.JobTypeAsync, + Priority: 100, + }) + if err != nil { + return fmt.Errorf("unable to publish async jobs to notifier with error %s", err.Error()) + } + + pendingJobsMap := make(map[int64]model.SourceJob) + for _, job := range pendingJobs { + pendingJobsMap[job.ID] = job + } + + select { + case <-ctx.Done(): + a.logger.Info("context cancelled, exiting") + return nil + case responses, ok := <-ch: + if !ok { + for _, job := range pendingJobsMap { + err := a.sourceRepo.OnUpdateFailure( + ctx, + job.ID, + errors.New("receiving channel closed"), + a.config.maxAttemptsPerJob, + ) + if err != nil { + return fmt.Errorf("unable to update async job with error %s", err.Error()) + } + } + continue + } + if responses.Err != nil { + for _, job := range pendingJobsMap { + err := a.sourceRepo.OnUpdateFailure( + ctx, + job.ID, + responses.Err, + a.config.maxAttemptsPerJob, + ) + if err != nil { + return fmt.Errorf("unable to update async job with error %s", err.Error()) + } + } + continue + } + + for _, job := range responses.Jobs { + var response notifierResponse + err := json.Unmarshal(job.Payload, &response) + if err != nil { + return fmt.Errorf("unable to unmarshal notifier response with error %s", err.Error()) + } + + if pj, ok := pendingJobsMap[response.Id]; ok { + pj.Status = job.Status + pj.Error = job.Error + } + } + + for _, job := range pendingJobsMap { + if job.Error != nil { + err := a.sourceRepo.OnUpdateFailure( + ctx, + job.ID, + responses.Err, + a.config.maxAttemptsPerJob, + ) + if err != nil { + return fmt.Errorf("unable to update async job with error %s", err.Error()) + } + continue + } + err := a.sourceRepo.OnUpdateSuccess( + ctx, + job.ID, + ) + if err != nil { + return fmt.Errorf("unable to update async job with error %s", err.Error()) + } + } + case <-time.After(a.config.processingTimeout): + for _, job := range pendingJobsMap { + err := a.sourceRepo.OnUpdateFailure( + ctx, + job.ID, + errors.New("job timed out"), + a.config.maxAttemptsPerJob, + ) + if err != nil { + return fmt.Errorf("unable to update async job with error %s", err.Error()) + } + } + } + } +} + +type SourceUploader struct{} + +func (*SourceUploader) IsWarehouseSchemaEmpty() bool { return true } +func (*SourceUploader) UpdateLocalSchema(context.Context, model.Schema) error { return nil } +func (*SourceUploader) GetTableSchemaInUpload(string) model.TableSchema { return model.TableSchema{} } +func (*SourceUploader) GetTableSchemaInWarehouse(string) model.TableSchema { + return model.TableSchema{} +} +func (*SourceUploader) ShouldOnDedupUseNewRecord() bool { return false } +func (*SourceUploader) UseRudderStorage() bool { return false } +func (*SourceUploader) CanAppend() bool { return false } +func (*SourceUploader) GetLoadFileGenStartTIme() time.Time { return time.Time{} } +func (*SourceUploader) GetLoadFileType() string { return "" } +func (*SourceUploader) GetFirstLastEvent() (time.Time, time.Time) { return time.Now(), time.Now() } +func (*SourceUploader) GetLocalSchema(context.Context) (model.Schema, error) { + return model.Schema{}, nil +} +func (*SourceUploader) GetSingleLoadFile(context.Context, string) (warehouseutils.LoadFile, error) { + return warehouseutils.LoadFile{}, nil +} +func (*SourceUploader) GetLoadFilesMetadata(context.Context, warehouseutils.GetLoadFilesOptions) []warehouseutils.LoadFile { + return []warehouseutils.LoadFile{} +} +func (*SourceUploader) GetSampleLoadFileLocation(context.Context, string) (string, error) { + return "", nil +} diff --git a/warehouse/warehouse.go b/warehouse/warehouse.go index 11cf97030e..45c354012f 100644 --- a/warehouse/warehouse.go +++ b/warehouse/warehouse.go @@ -34,8 +34,8 @@ import ( "github.com/rudderlabs/rudder-server/utils/misc" "github.com/rudderlabs/rudder-server/utils/types" "github.com/rudderlabs/rudder-server/warehouse/archive" - "github.com/rudderlabs/rudder-server/warehouse/jobs" "github.com/rudderlabs/rudder-server/warehouse/multitenant" + "github.com/rudderlabs/rudder-server/warehouse/source" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" ) @@ -56,7 +56,7 @@ type App struct { constraintsManager *constraintsManager encodingFactory *encoding.Factory fileManagerFactory filemanager.Factory - sourcesManager *jobs.AsyncJobWh + sourcesManager *source.Manager admin *Admin triggerStore *sync.Map @@ -155,12 +155,12 @@ func (a *App) Setup(ctx context.Context) error { return fmt.Errorf("cannot setup notifier: %w", err) } - a.sourcesManager = jobs.New( - ctx, + a.sourcesManager = source.New( + a.conf, + a.logger, a.db, a.notifier, ) - jobs.WithConfig(a.sourcesManager, a.conf) a.grpcServer, err = NewGRPCServer( a.conf, @@ -391,7 +391,7 @@ func (a *App) Run(ctx context.Context) error { return nil }) g.Go(misc.WithBugsnagForWarehouse(func() error { - return a.sourcesManager.Run() + return a.sourcesManager.Run(gCtx) })) } From 21f6af8ac0c373f393715ea59aa60fd7d8337dec Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Sun, 15 Oct 2023 02:19:02 +0530 Subject: [PATCH 2/7] chore: some more changes --- rudder-transformer | 1 - services/notifier/notifier.go | 10 +- warehouse/api/http.go | 10 +- warehouse/api/http_test.go | 7 +- warehouse/internal/repo/source.go | 22 ++- warehouse/internal/repo/source_test.go | 172 +++++++++++++++++++++++- warehouse/internal/repo/table_upload.go | 11 +- warehouse/source/http_test.go | 24 ---- warehouse/source/source.go | 3 +- 9 files changed, 206 insertions(+), 54 deletions(-) delete mode 160000 rudder-transformer diff --git a/rudder-transformer b/rudder-transformer deleted file mode 160000 index 4a38b158de..0000000000 --- a/rudder-transformer +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 4a38b158de4ed9c5db88d83d0578c03521c80e58 diff --git a/services/notifier/notifier.go b/services/notifier/notifier.go index 377c52ae90..6f7dee9469 100644 --- a/services/notifier/notifier.go +++ b/services/notifier/notifier.go @@ -39,7 +39,7 @@ const ( JobTypeAsync JobType = "async_job" ) -type JobStatus = string +type JobStatus string const ( Waiting JobStatus = "waiting" @@ -187,11 +187,11 @@ func New( }) n.stats.claimSucceeded = n.statsFactory.NewTaggedStat("pgnotifier.claim", stats.CountType, stats.Tags{ "module": module, - "status": Succeeded, + "status": string(Succeeded), }) n.stats.claimFailed = n.statsFactory.NewTaggedStat("pgnotifier.claim", stats.CountType, stats.Tags{ "module": module, - "status": Failed, + "status": string(Failed), }) n.stats.claimUpdateFailed = n.statsFactory.NewStat("pgnotifier.claimUpdateFailed", stats.CountType) n.stats.publishTime = n.statsFactory.NewTaggedStat("pgnotifier.publishTime", stats.TimerType, stats.Tags{ @@ -199,11 +199,11 @@ func New( }) n.stats.claimSucceededTime = n.statsFactory.NewTaggedStat("pgnotifier.claimTime", stats.TimerType, stats.Tags{ "module": module, - "status": Succeeded, + "status": string(Succeeded), }) n.stats.claimFailedTime = n.statsFactory.NewTaggedStat("pgnotifier.claimTime", stats.TimerType, stats.Tags{ "module": module, - "status": Failed, + "status": string(Failed), }) n.stats.abortedRecords = n.statsFactory.NewTaggedStat("pg_notifier.aborted_records", stats.CountType, stats.Tags{ "workspace": n.workspaceIdentifier, diff --git a/warehouse/api/http.go b/warehouse/api/http.go index e945b2b8f3..d5e69a497e 100644 --- a/warehouse/api/http.go +++ b/warehouse/api/http.go @@ -74,7 +74,7 @@ type Api struct { bcConfig backendconfig.BackendConfig tenantManager *multitenant.Manager bcManager *bcm.BackendConfigManager - asyncManager *source.Manager + sourceManager *source.Manager stagingRepo *repo.StagingFiles uploadRepo *repo.Uploads schemaRepo *repo.WHSchema @@ -99,7 +99,7 @@ func NewApi( notifier *notifier.Notifier, tenantManager *multitenant.Manager, bcManager *bcm.BackendConfigManager, - asyncManager *source.Manager, + sourceManager *source.Manager, triggerStore *sync.Map, ) *Api { a := &Api{ @@ -111,7 +111,7 @@ func NewApi( statsFactory: statsFactory, tenantManager: tenantManager, bcManager: bcManager, - asyncManager: asyncManager, + sourceManager: sourceManager, triggerStore: triggerStore, stagingRepo: repo.NewStagingFiles(db), uploadRepo: repo.NewUploads(db), @@ -166,8 +166,8 @@ func (a *Api) addMasterEndpoints(ctx context.Context, r chi.Router) { r.Post("/pending-events", a.logMiddleware(a.pendingEventsHandler)) r.Post("/trigger-upload", a.logMiddleware(a.triggerUploadHandler)) - r.Post("/jobs", a.logMiddleware(a.asyncManager.InsertJobHandler)) // TODO: add degraded mode - r.Get("/jobs/status", a.logMiddleware(a.asyncManager.StatusJobHandler)) // TODO: add degraded mode + r.Post("/jobs", a.logMiddleware(a.sourceManager.InsertJobHandler)) // TODO: add degraded mode + r.Get("/jobs/status", a.logMiddleware(a.sourceManager.StatusJobHandler)) // TODO: add degraded mode r.Get("/fetch-tables", a.logMiddleware(a.fetchTablesHandler)) // TODO: Remove this endpoint once sources change is released }) diff --git a/warehouse/api/http_test.go b/warehouse/api/http_test.go index 29d2a8330e..3c6d4a8e79 100644 --- a/warehouse/api/http_test.go +++ b/warehouse/api/http_test.go @@ -187,7 +187,12 @@ func TestHTTPApi(t *testing.T) { err = n.Setup(ctx, pgResource.DBDsn) require.NoError(t, err) - sourcesManager := source.New(config.New(), logger.NOP, db, n) + sourcesManager := source.New( + config.Default, + logger.NOP, + db, + n, + ) g, gCtx := errgroup.WithContext(ctx) g.Go(func() error { diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go index 768662957c..8991418fc5 100644 --- a/warehouse/internal/repo/source.go +++ b/warehouse/internal/repo/source.go @@ -84,7 +84,7 @@ func (repo *Source) Insert( model.SourceJobStatusWaiting, repo.now(), repo.now(), - sourceJob.Status, + sourceJob.JobType, sourceJob.WorkspaceID, sourceJob.Metadata, ).Scan(&id) @@ -237,7 +237,7 @@ func (repo *Source) OnUpdateSuccess( ctx context.Context, id int64, ) error { - _, err := repo.db.ExecContext(ctx, ` + r, err := repo.db.ExecContext(ctx, ` UPDATE `+sourceJobTableName+` SET @@ -253,6 +253,14 @@ func (repo *Source) OnUpdateSuccess( if err != nil { return fmt.Errorf("on update success: %w", err) } + rowsAffected, err := r.RowsAffected() + if err != nil { + return fmt.Errorf("rows affected: %w", err) + } + if rowsAffected == 0 { + return fmt.Errorf("no rows affected") + } + return nil } @@ -262,7 +270,7 @@ func (repo *Source) OnUpdateFailure( error error, maxAttempt int, ) error { - _, err := repo.db.ExecContext(ctx, ` + r, err := repo.db.ExecContext(ctx, ` UPDATE `+sourceJobTableName+` SET @@ -284,5 +292,13 @@ func (repo *Source) OnUpdateFailure( if err != nil { return fmt.Errorf("on update success: %w", err) } + rowsAffected, err := r.RowsAffected() + if err != nil { + return fmt.Errorf("rows affected: %w", err) + } + if rowsAffected == 0 { + return fmt.Errorf("no rows affected") + } + return nil } diff --git a/warehouse/internal/repo/source_test.go b/warehouse/internal/repo/source_test.go index 2133a1f8ea..e2092553b9 100644 --- a/warehouse/internal/repo/source_test.go +++ b/warehouse/internal/repo/source_test.go @@ -3,6 +3,7 @@ package repo_test import ( "context" "encoding/json" + "errors" "strconv" "testing" "time" @@ -17,6 +18,7 @@ import ( func sourcesJobs( sourceID, destinationID, workspaceID string, + jobType string, metadata json.RawMessage, count int, ) []model.SourceJob { sourcesJobs := make([]model.SourceJob, 0, count) @@ -26,7 +28,8 @@ func sourcesJobs( DestinationID: destinationID, TableName: "table" + strconv.Itoa(i), WorkspaceID: workspaceID, - Metadata: json.RawMessage(`{"key": "value"}`), + Metadata: metadata, + JobType: jobType, }) } return sourcesJobs @@ -49,7 +52,7 @@ func TestSource_Insert(t *testing.T) { t.Run("success", func(t *testing.T) { ids, err := repoSource.Insert( ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 10), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 10), ) require.NoError(t, err) require.Len(t, ids, 10) @@ -59,7 +62,7 @@ func TestSource_Insert(t *testing.T) { cancel() ids, err := repoSource.Insert(ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 1), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 1), ) require.ErrorIs(t, err, context.Canceled) require.Nil(t, ids) @@ -82,7 +85,7 @@ func TestSource_Reset(t *testing.T) { t.Run("success", func(t *testing.T) { ids, err := repoSource.Insert(ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 10), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 10), ) require.NoError(t, err) require.Len(t, ids, 10) @@ -125,3 +128,164 @@ func TestSource_Reset(t *testing.T) { require.ErrorIs(t, err, context.Canceled) }) } + +func TestSource_GetToProcess(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 25), + ) + require.NoError(t, err) + require.Len(t, ids, 25) + + t.Run("less jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 25) + require.NoError(t, err) + require.Len(t, jobs, 25) + }) + t.Run("more jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 20) + require.NoError(t, err) + require.Len(t, jobs, 20) + }) + t.Run("equal jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 50) + require.NoError(t, err) + require.Len(t, jobs, 25) + }) + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + jobs, err := repoSource.GetToProcess(ctx, 10) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, jobs) + }) +} + +func TestSource_GetByJobRunTaskRun(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + jobRun = "test-job-run" + taskRun = "test-task-run" + otherJobRun = "other-job-run" + otherTaskRun = "other-task-run" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("job is available", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), 1), + ) + require.NoError(t, err) + require.Len(t, ids, 1) + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.NoError(t, err) + require.Equal(t, job, &model.SourceJob{ + ID: 1, + SourceID: sourceId, + DestinationID: destinationId, + WorkspaceID: workspaceId, + TableName: "table0", + Status: model.SourceJobStatusWaiting, + Error: nil, + JobType: model.DeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 0, + }) + }) + t.Run("job is not available", func(t *testing.T) { + job, err := repoSource.GetByJobRunTaskRun(ctx, otherJobRun, otherTaskRun) + require.ErrorIs(t, err, model.ErrSourcesJobNotFound) + require.Nil(t, job) + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, job) + }) +} + +func TestSource_OnUpdateSuccess(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + jobRun = "test-job-run" + taskRun = "test-task-run" + otherJobRun = "other-job-run" + otherTaskRun = "other-task-run" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) {}) + t.Run("job not found", func(t *testing.T) {}) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + err := repoSource.OnUpdateSuccess(ctx, int64(1)) + require.ErrorIs(t, err, context.Canceled) + }) +} + +func TestSource_OnUpdateFailure(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + testError = "test-error" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) {}) + t.Run("job not found", func(t *testing.T) {}) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + err := repoSource.OnUpdateFailure(ctx, int64(1), errors.New(testError), 1) + require.ErrorIs(t, err, context.Canceled) + }) +} diff --git a/warehouse/internal/repo/table_upload.go b/warehouse/internal/repo/table_upload.go index 80fcc820b2..e0fbd550d7 100644 --- a/warehouse/internal/repo/table_upload.go +++ b/warehouse/internal/repo/table_upload.go @@ -91,15 +91,7 @@ func (repo *TableUploads) Insert(ctx context.Context, uploadID int64, tableNames defer func() { _ = stmt.Close() }() for _, tableName := range tableNames { - _, err = stmt.ExecContext( - ctx, - uploadID, - tableName, - model.TableUploadWaiting, - "{}", - repo.now(), - repo.now(), - ) + _, err = stmt.ExecContext(ctx, uploadID, tableName, model.TableUploadWaiting, "{}", repo.now(), repo.now()) if err != nil { return fmt.Errorf(`stmt exec: %w`, err) } @@ -120,6 +112,7 @@ func (repo *TableUploads) GetByUploadID(ctx context.Context, uploadID int64) ([] if err != nil { return nil, fmt.Errorf("querying table uploads: %w", err) } + defer func() { _ = rows.Close() }() var tableUploads []model.TableUpload for rows.Next() { diff --git a/warehouse/source/http_test.go b/warehouse/source/http_test.go index 70d497e015..dafd62abe1 100644 --- a/warehouse/source/http_test.go +++ b/warehouse/source/http_test.go @@ -199,18 +199,6 @@ func TestAsyncJobHandlers(t *testing.T) { }) t.Run("InsertJobHandler", func(t *testing.T) { - t.Run("Not enabled", func(t *testing.T) { - req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", nil) - resp := httptest.NewRecorder() - - jobsManager := New(config.New(), logger.NOP, db, n) - jobsManager.InsertJobHandler(resp, req) - require.Equal(t, http.StatusInternalServerError, resp.Code) - - b, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, "warehouse jobs api not initialized\n", string(b)) - }) t.Run("invalid payload", func(t *testing.T) { req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", bytes.NewReader([]byte(`"Invalid payload"`))) resp := httptest.NewRecorder() @@ -262,18 +250,6 @@ func TestAsyncJobHandlers(t *testing.T) { }) t.Run("StatusJobHandler", func(t *testing.T) { - t.Run("Not enabled", func(t *testing.T) { - req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) - resp := httptest.NewRecorder() - - jobsManager := New(config.New(), logger.NOP, db, n) - jobsManager.StatusJobHandler(resp, req) - require.Equal(t, http.StatusInternalServerError, resp.Code) - - b, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, "warehouse jobs api not initialized\n", string(b)) - }) t.Run("invalid payload", func(t *testing.T) { req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) resp := httptest.NewRecorder() diff --git a/warehouse/source/source.go b/warehouse/source/source.go index 8d78ab2b60..9979923472 100644 --- a/warehouse/source/source.go +++ b/warehouse/source/source.go @@ -37,7 +37,6 @@ type Manager struct { } } -// New Initializes AsyncJobWh structure with appropriate variabless func New( conf *config.Config, log logger.Logger, @@ -155,7 +154,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { } if pj, ok := pendingJobsMap[response.Id]; ok { - pj.Status = job.Status + pj.Status = string(job.Status) pj.Error = job.Error } } From b5db1405cb6bf3c05b01bfe8c200ad9c7c965f24 Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Sun, 15 Oct 2023 02:19:02 +0530 Subject: [PATCH 3/7] chore: some more changes --- rudder-transformer | 1 - services/notifier/notifier.go | 10 +- warehouse/api/http.go | 10 +- warehouse/api/http_test.go | 7 +- .../integrations/bigquery/bigquery_test.go | 10 +- warehouse/integrations/mssql/mssql_test.go | 10 +- .../integrations/postgres/postgres_test.go | 10 +- .../integrations/redshift/redshift_test.go | 10 +- .../integrations/snowflake/snowflake_test.go | 10 +- warehouse/integrations/testhelper/setup.go | 14 +- warehouse/integrations/testhelper/verify.go | 24 +-- warehouse/internal/repo/source.go | 22 ++- warehouse/internal/repo/source_test.go | 159 +++++++++++++++++- warehouse/internal/repo/table_upload.go | 11 +- warehouse/slave/worker.go | 26 +-- warehouse/slave/worker_test.go | 8 +- warehouse/source/http.go | 18 +- warehouse/source/http_test.go | 26 +-- warehouse/source/source.go | 21 ++- 19 files changed, 273 insertions(+), 134 deletions(-) delete mode 160000 rudder-transformer diff --git a/rudder-transformer b/rudder-transformer deleted file mode 160000 index 4a38b158de..0000000000 --- a/rudder-transformer +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 4a38b158de4ed9c5db88d83d0578c03521c80e58 diff --git a/services/notifier/notifier.go b/services/notifier/notifier.go index 377c52ae90..6f7dee9469 100644 --- a/services/notifier/notifier.go +++ b/services/notifier/notifier.go @@ -39,7 +39,7 @@ const ( JobTypeAsync JobType = "async_job" ) -type JobStatus = string +type JobStatus string const ( Waiting JobStatus = "waiting" @@ -187,11 +187,11 @@ func New( }) n.stats.claimSucceeded = n.statsFactory.NewTaggedStat("pgnotifier.claim", stats.CountType, stats.Tags{ "module": module, - "status": Succeeded, + "status": string(Succeeded), }) n.stats.claimFailed = n.statsFactory.NewTaggedStat("pgnotifier.claim", stats.CountType, stats.Tags{ "module": module, - "status": Failed, + "status": string(Failed), }) n.stats.claimUpdateFailed = n.statsFactory.NewStat("pgnotifier.claimUpdateFailed", stats.CountType) n.stats.publishTime = n.statsFactory.NewTaggedStat("pgnotifier.publishTime", stats.TimerType, stats.Tags{ @@ -199,11 +199,11 @@ func New( }) n.stats.claimSucceededTime = n.statsFactory.NewTaggedStat("pgnotifier.claimTime", stats.TimerType, stats.Tags{ "module": module, - "status": Succeeded, + "status": string(Succeeded), }) n.stats.claimFailedTime = n.statsFactory.NewTaggedStat("pgnotifier.claimTime", stats.TimerType, stats.Tags{ "module": module, - "status": Failed, + "status": string(Failed), }) n.stats.abortedRecords = n.statsFactory.NewTaggedStat("pg_notifier.aborted_records", stats.CountType, stats.Tags{ "workspace": n.workspaceIdentifier, diff --git a/warehouse/api/http.go b/warehouse/api/http.go index e945b2b8f3..d5e69a497e 100644 --- a/warehouse/api/http.go +++ b/warehouse/api/http.go @@ -74,7 +74,7 @@ type Api struct { bcConfig backendconfig.BackendConfig tenantManager *multitenant.Manager bcManager *bcm.BackendConfigManager - asyncManager *source.Manager + sourceManager *source.Manager stagingRepo *repo.StagingFiles uploadRepo *repo.Uploads schemaRepo *repo.WHSchema @@ -99,7 +99,7 @@ func NewApi( notifier *notifier.Notifier, tenantManager *multitenant.Manager, bcManager *bcm.BackendConfigManager, - asyncManager *source.Manager, + sourceManager *source.Manager, triggerStore *sync.Map, ) *Api { a := &Api{ @@ -111,7 +111,7 @@ func NewApi( statsFactory: statsFactory, tenantManager: tenantManager, bcManager: bcManager, - asyncManager: asyncManager, + sourceManager: sourceManager, triggerStore: triggerStore, stagingRepo: repo.NewStagingFiles(db), uploadRepo: repo.NewUploads(db), @@ -166,8 +166,8 @@ func (a *Api) addMasterEndpoints(ctx context.Context, r chi.Router) { r.Post("/pending-events", a.logMiddleware(a.pendingEventsHandler)) r.Post("/trigger-upload", a.logMiddleware(a.triggerUploadHandler)) - r.Post("/jobs", a.logMiddleware(a.asyncManager.InsertJobHandler)) // TODO: add degraded mode - r.Get("/jobs/status", a.logMiddleware(a.asyncManager.StatusJobHandler)) // TODO: add degraded mode + r.Post("/jobs", a.logMiddleware(a.sourceManager.InsertJobHandler)) // TODO: add degraded mode + r.Get("/jobs/status", a.logMiddleware(a.sourceManager.StatusJobHandler)) // TODO: add degraded mode r.Get("/fetch-tables", a.logMiddleware(a.fetchTablesHandler)) // TODO: Remove this endpoint once sources change is released }) diff --git a/warehouse/api/http_test.go b/warehouse/api/http_test.go index 29d2a8330e..3c6d4a8e79 100644 --- a/warehouse/api/http_test.go +++ b/warehouse/api/http_test.go @@ -187,7 +187,12 @@ func TestHTTPApi(t *testing.T) { err = n.Setup(ctx, pgResource.DBDsn) require.NoError(t, err) - sourcesManager := source.New(config.New(), logger.NOP, db, n) + sourcesManager := source.New( + config.Default, + logger.NOP, + db, + n, + ) g, gCtx := errgroup.WithContext(ctx) g.Go(func() error { diff --git a/warehouse/integrations/bigquery/bigquery_test.go b/warehouse/integrations/bigquery/bigquery_test.go index 0af0d31253..568aab300a 100644 --- a/warehouse/integrations/bigquery/bigquery_test.go +++ b/warehouse/integrations/bigquery/bigquery_test.go @@ -159,7 +159,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap testhelper.EventsCountMap tableUploadsEventsMap testhelper.EventsCountMap warehouseEventsMap testhelper.EventsCountMap - asyncJob bool + sourceJob bool skipModifiedEvents bool prerequisite func(t testing.TB) isDedupEnabled bool @@ -189,7 +189,7 @@ func TestIntegration(t *testing.T) { stagingFilePrefix: "testdata/upload-job-merge-mode", }, { - name: "Async Job", + name: "Source Job", writeKey: sourcesWriteKey, sourceID: sourcesSourceID, destinationID: sourcesDestinationID, @@ -204,7 +204,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap: testhelper.SourcesLoadFilesEventsMap(), tableUploadsEventsMap: testhelper.SourcesTableUploadsEventsMap(), warehouseEventsMap: testhelper.SourcesWarehouseEventsMap(), - asyncJob: true, + sourceJob: true, isDedupEnabled: false, prerequisite: func(t testing.TB) { t.Helper() @@ -341,7 +341,7 @@ func TestIntegration(t *testing.T) { LoadFilesEventsMap: tc.loadFilesEventsMap, TableUploadsEventsMap: tc.tableUploadsEventsMap, WarehouseEventsMap: tc.warehouseEventsMap, - AsyncJob: tc.asyncJob, + SourceJob: tc.sourceJob, Config: conf, WorkspaceID: workspaceID, DestinationType: destType, @@ -353,7 +353,7 @@ func TestIntegration(t *testing.T) { StagingFilePath: tc.stagingFilePrefix + ".staging-2.json", UserID: testhelper.GetUserId(destType), } - if tc.asyncJob { + if tc.sourceJob { ts2.UserID = ts1.UserID } ts2.VerifyEvents(t) diff --git a/warehouse/integrations/mssql/mssql_test.go b/warehouse/integrations/mssql/mssql_test.go index 7d4bd15953..5e4a1081b2 100644 --- a/warehouse/integrations/mssql/mssql_test.go +++ b/warehouse/integrations/mssql/mssql_test.go @@ -160,7 +160,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap testhelper.EventsCountMap tableUploadsEventsMap testhelper.EventsCountMap warehouseEventsMap testhelper.EventsCountMap - asyncJob bool + sourceJob bool stagingFilePrefix string }{ { @@ -173,7 +173,7 @@ func TestIntegration(t *testing.T) { stagingFilePrefix: "testdata/upload-job", }, { - name: "Async Job", + name: "Source Job", writeKey: sourcesWriteKey, schema: sourcesNamespace, tables: []string{"tracks", "google_sheet"}, @@ -183,7 +183,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap: testhelper.SourcesLoadFilesEventsMap(), tableUploadsEventsMap: testhelper.SourcesTableUploadsEventsMap(), warehouseEventsMap: testhelper.SourcesWarehouseEventsMap(), - asyncJob: true, + sourceJob: true, stagingFilePrefix: "testdata/sources-job", }, } @@ -244,7 +244,7 @@ func TestIntegration(t *testing.T) { LoadFilesEventsMap: tc.loadFilesEventsMap, TableUploadsEventsMap: tc.tableUploadsEventsMap, WarehouseEventsMap: tc.warehouseEventsMap, - AsyncJob: tc.asyncJob, + SourceJob: tc.sourceJob, Config: conf, WorkspaceID: workspaceID, DestinationType: destType, @@ -256,7 +256,7 @@ func TestIntegration(t *testing.T) { StagingFilePath: tc.stagingFilePrefix + ".staging-2.json", UserID: testhelper.GetUserId(destType), } - if tc.asyncJob { + if tc.sourceJob { ts2.UserID = ts1.UserID } ts2.VerifyEvents(t) diff --git a/warehouse/integrations/postgres/postgres_test.go b/warehouse/integrations/postgres/postgres_test.go index 3b41bf6163..ed60037565 100644 --- a/warehouse/integrations/postgres/postgres_test.go +++ b/warehouse/integrations/postgres/postgres_test.go @@ -196,7 +196,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap testhelper.EventsCountMap tableUploadsEventsMap testhelper.EventsCountMap warehouseEventsMap testhelper.EventsCountMap - asyncJob bool + sourceJob bool stagingFilePrefix string }{ { @@ -211,7 +211,7 @@ func TestIntegration(t *testing.T) { stagingFilePrefix: "testdata/upload-job", }, { - name: "Async Job", + name: "Source Job", writeKey: sourcesWriteKey, schema: sourcesNamespace, tables: []string{"tracks", "google_sheet"}, @@ -221,7 +221,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap: testhelper.SourcesLoadFilesEventsMap(), tableUploadsEventsMap: testhelper.SourcesTableUploadsEventsMap(), warehouseEventsMap: testhelper.SourcesWarehouseEventsMap(), - asyncJob: true, + sourceJob: true, stagingFilePrefix: "testdata/sources-job", }, } @@ -282,7 +282,7 @@ func TestIntegration(t *testing.T) { LoadFilesEventsMap: tc.loadFilesEventsMap, TableUploadsEventsMap: tc.tableUploadsEventsMap, WarehouseEventsMap: tc.warehouseEventsMap, - AsyncJob: tc.asyncJob, + SourceJob: tc.sourceJob, Config: conf, WorkspaceID: workspaceID, DestinationType: destType, @@ -294,7 +294,7 @@ func TestIntegration(t *testing.T) { StagingFilePath: tc.stagingFilePrefix + ".staging-2.json", UserID: testhelper.GetUserId(destType), } - if tc.asyncJob { + if tc.sourceJob { ts2.UserID = ts1.UserID } ts2.VerifyEvents(t) diff --git a/warehouse/integrations/redshift/redshift_test.go b/warehouse/integrations/redshift/redshift_test.go index b1bc288af1..63804c7075 100644 --- a/warehouse/integrations/redshift/redshift_test.go +++ b/warehouse/integrations/redshift/redshift_test.go @@ -197,7 +197,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap testhelper.EventsCountMap tableUploadsEventsMap testhelper.EventsCountMap warehouseEventsMap testhelper.EventsCountMap - asyncJob bool + sourceJob bool stagingFilePrefix string }{ { @@ -210,7 +210,7 @@ func TestIntegration(t *testing.T) { stagingFilePrefix: "testdata/upload-job", }, { - name: "Async Job", + name: "Source Job", writeKey: sourcesWriteKey, schema: sourcesNamespace, tables: []string{"tracks", "google_sheet"}, @@ -220,7 +220,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap: testhelper.SourcesLoadFilesEventsMap(), tableUploadsEventsMap: testhelper.SourcesTableUploadsEventsMap(), warehouseEventsMap: testhelper.SourcesWarehouseEventsMap(), - asyncJob: true, + sourceJob: true, stagingFilePrefix: "testdata/sources-job", }, } @@ -292,7 +292,7 @@ func TestIntegration(t *testing.T) { LoadFilesEventsMap: tc.loadFilesEventsMap, TableUploadsEventsMap: tc.tableUploadsEventsMap, WarehouseEventsMap: tc.warehouseEventsMap, - AsyncJob: tc.asyncJob, + SourceJob: tc.sourceJob, Config: conf, WorkspaceID: workspaceID, DestinationType: destType, @@ -304,7 +304,7 @@ func TestIntegration(t *testing.T) { StagingFilePath: tc.stagingFilePrefix + ".staging-1.json", UserID: testhelper.GetUserId(destType), } - if tc.asyncJob { + if tc.sourceJob { ts2.UserID = ts1.UserID } ts2.VerifyEvents(t) diff --git a/warehouse/integrations/snowflake/snowflake_test.go b/warehouse/integrations/snowflake/snowflake_test.go index 51476b186e..92712c3da2 100644 --- a/warehouse/integrations/snowflake/snowflake_test.go +++ b/warehouse/integrations/snowflake/snowflake_test.go @@ -231,7 +231,7 @@ func TestIntegration(t *testing.T) { warehouseEventsMap2 testhelper.EventsCountMap cred *testCredentials database string - asyncJob bool + sourceJob bool stagingFilePrefix string emptyJobRunID bool appendMode bool @@ -295,7 +295,7 @@ func TestIntegration(t *testing.T) { stagingFilePrefix: "testdata/upload-job-case-sensitive", }, { - name: "Async Job with Sources", + name: "Source Job with Sources", writeKey: sourcesWriteKey, schema: sourcesNamespace, tables: []string{"tracks", "google_sheet"}, @@ -312,7 +312,7 @@ func TestIntegration(t *testing.T) { loadFilesEventsMap: testhelper.SourcesLoadFilesEventsMap(), tableUploadsEventsMap: testhelper.SourcesTableUploadsEventsMap(), warehouseEventsMap: testhelper.SourcesWarehouseEventsMap(), - asyncJob: true, + sourceJob: true, stagingFilePrefix: "testdata/sources-job", }, { @@ -442,7 +442,7 @@ func TestIntegration(t *testing.T) { LoadFilesEventsMap: tc.loadFilesEventsMap, TableUploadsEventsMap: tc.tableUploadsEventsMap, WarehouseEventsMap: whEventsMap, - AsyncJob: tc.asyncJob, + SourceJob: tc.sourceJob, Config: conf, WorkspaceID: workspaceID, DestinationType: destType, @@ -454,7 +454,7 @@ func TestIntegration(t *testing.T) { StagingFilePath: tc.stagingFilePrefix + ".staging-2.json", UserID: userID, } - if tc.asyncJob { + if tc.sourceJob { ts2.UserID = ts1.UserID } ts2.VerifyEvents(t) diff --git a/warehouse/integrations/testhelper/setup.go b/warehouse/integrations/testhelper/setup.go index 91523e0b7f..247a9e4474 100644 --- a/warehouse/integrations/testhelper/setup.go +++ b/warehouse/integrations/testhelper/setup.go @@ -29,10 +29,10 @@ import ( ) const ( - WaitFor2Minute = 2 * time.Minute - WaitFor10Minute = 10 * time.Minute - DefaultQueryFrequency = 100 * time.Millisecond - AsyncJOBQueryFrequency = 1000 * time.Millisecond + WaitFor2Minute = 2 * time.Minute + WaitFor10Minute = 10 * time.Minute + DefaultQueryFrequency = 100 * time.Millisecond + SourceJobQueryFrequency = 1000 * time.Millisecond ) const ( @@ -64,7 +64,7 @@ type TestConfig struct { TableUploadsEventsMap EventsCountMap WarehouseEventsMap EventsCountMap JobsDB *sql.DB - AsyncJob bool + SourceJob bool SkipWarehouse bool HTTPPort int } @@ -80,8 +80,8 @@ func (w *TestConfig) VerifyEvents(t testing.TB) { verifyEventsInLoadFiles(t, w) verifyEventsInTableUploads(t, w) - if w.AsyncJob { - verifyAsyncJob(t, w) + if w.SourceJob { + verifySourceJob(t, w) } if !w.SkipWarehouse { verifyEventsInWareHouse(t, w) diff --git a/warehouse/integrations/testhelper/verify.go b/warehouse/integrations/testhelper/verify.go index c7ce9d122d..3b8540cc57 100644 --- a/warehouse/integrations/testhelper/verify.go +++ b/warehouse/integrations/testhelper/verify.go @@ -236,14 +236,14 @@ func queryCount(cl *whclient.Client, statement string) (int64, error) { return strconv.ParseInt(result.Values[0][0], 10, 64) } -func verifyAsyncJob(t testing.TB, tc *TestConfig) { +func verifySourceJob(t testing.TB, tc *TestConfig) { t.Helper() - t.Logf("Creating async job for sourceID: %s, jobRunID: %s, taskRunID: %s, destinationID: %s, workspaceID: %s", + t.Logf("Creating source job for sourceID: %s, jobRunID: %s, taskRunID: %s, destinationID: %s, workspaceID: %s", tc.SourceID, tc.JobRunID, tc.TaskRunID, tc.DestinationID, tc.WorkspaceID, ) - asyncPayload := fmt.Sprintf( + payload := fmt.Sprintf( `{ "source_id":"%s","job_run_id":"%s","task_run_id":"%s","channel":"sources", "async_job_type":"deletebyjobrunid","destination_id":"%s","start_time":"%s","workspace_id":"%s" @@ -257,7 +257,7 @@ func verifyAsyncJob(t testing.TB, tc *TestConfig) { ) url := fmt.Sprintf("http://localhost:%d/v1/warehouse/jobs", tc.HTTPPort) - req, err := http.NewRequest(http.MethodPost, url, strings.NewReader(asyncPayload)) + req, err := http.NewRequest(http.MethodPost, url, strings.NewReader(payload)) require.NoError(t, err) req.Header.Add("Content-Type", "application/json") @@ -275,11 +275,11 @@ func verifyAsyncJob(t testing.TB, tc *TestConfig) { require.NoError(t, err) require.Equal(t, res.Status, "200 OK") - t.Logf("Verify async job status for sourceID: %s, jobRunID: %s, taskRunID: %s, destID: %s, workspaceID: %s", + t.Logf("Verify source job status for sourceID: %s, jobRunID: %s, taskRunID: %s, destID: %s, workspaceID: %s", tc.SourceID, tc.JobRunID, tc.TaskRunID, tc.DestinationID, tc.WorkspaceID, ) - type asyncResponse struct { + type jobResponse struct { Status string `json:"status"` Error string `json:"error"` } @@ -309,14 +309,14 @@ func verifyAsyncJob(t testing.TB, tc *TestConfig) { return false } - var asyncRes asyncResponse - if err = json.NewDecoder(res.Body).Decode(&asyncRes); err != nil { + var jr jobResponse + if err = json.NewDecoder(res.Body).Decode(&jr); err != nil { return false } - return asyncRes.Status == "succeeded" + return jr.Status == "succeeded" } - require.Eventuallyf(t, operation, WaitFor10Minute, AsyncJOBQueryFrequency, - "Failed to get async job status for job_run_id: %s, task_run_id: %s, source_id: %s, destination_id: %s: %v", + require.Eventuallyf(t, operation, WaitFor10Minute, SourceJobQueryFrequency, + "Failed to get source job status for job_run_id: %s, task_run_id: %s, source_id: %s, destination_id: %s: %v", tc.JobRunID, tc.TaskRunID, tc.SourceID, @@ -324,7 +324,7 @@ func verifyAsyncJob(t testing.TB, tc *TestConfig) { err, ) - t.Logf("Completed verifying async job") + t.Logf("Completed verifying source job") } func VerifyConfigurationTest(t testing.TB, destination backendconfig.DestinationT) { diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go index 768662957c..8991418fc5 100644 --- a/warehouse/internal/repo/source.go +++ b/warehouse/internal/repo/source.go @@ -84,7 +84,7 @@ func (repo *Source) Insert( model.SourceJobStatusWaiting, repo.now(), repo.now(), - sourceJob.Status, + sourceJob.JobType, sourceJob.WorkspaceID, sourceJob.Metadata, ).Scan(&id) @@ -237,7 +237,7 @@ func (repo *Source) OnUpdateSuccess( ctx context.Context, id int64, ) error { - _, err := repo.db.ExecContext(ctx, ` + r, err := repo.db.ExecContext(ctx, ` UPDATE `+sourceJobTableName+` SET @@ -253,6 +253,14 @@ func (repo *Source) OnUpdateSuccess( if err != nil { return fmt.Errorf("on update success: %w", err) } + rowsAffected, err := r.RowsAffected() + if err != nil { + return fmt.Errorf("rows affected: %w", err) + } + if rowsAffected == 0 { + return fmt.Errorf("no rows affected") + } + return nil } @@ -262,7 +270,7 @@ func (repo *Source) OnUpdateFailure( error error, maxAttempt int, ) error { - _, err := repo.db.ExecContext(ctx, ` + r, err := repo.db.ExecContext(ctx, ` UPDATE `+sourceJobTableName+` SET @@ -284,5 +292,13 @@ func (repo *Source) OnUpdateFailure( if err != nil { return fmt.Errorf("on update success: %w", err) } + rowsAffected, err := r.RowsAffected() + if err != nil { + return fmt.Errorf("rows affected: %w", err) + } + if rowsAffected == 0 { + return fmt.Errorf("no rows affected") + } + return nil } diff --git a/warehouse/internal/repo/source_test.go b/warehouse/internal/repo/source_test.go index 2133a1f8ea..8af6f98ef3 100644 --- a/warehouse/internal/repo/source_test.go +++ b/warehouse/internal/repo/source_test.go @@ -3,6 +3,7 @@ package repo_test import ( "context" "encoding/json" + "errors" "strconv" "testing" "time" @@ -17,6 +18,7 @@ import ( func sourcesJobs( sourceID, destinationID, workspaceID string, + jobType string, metadata json.RawMessage, count int, ) []model.SourceJob { sourcesJobs := make([]model.SourceJob, 0, count) @@ -26,7 +28,8 @@ func sourcesJobs( DestinationID: destinationID, TableName: "table" + strconv.Itoa(i), WorkspaceID: workspaceID, - Metadata: json.RawMessage(`{"key": "value"}`), + Metadata: metadata, + JobType: jobType, }) } return sourcesJobs @@ -49,7 +52,7 @@ func TestSource_Insert(t *testing.T) { t.Run("success", func(t *testing.T) { ids, err := repoSource.Insert( ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 10), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 10), ) require.NoError(t, err) require.Len(t, ids, 10) @@ -59,7 +62,7 @@ func TestSource_Insert(t *testing.T) { cancel() ids, err := repoSource.Insert(ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 1), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 1), ) require.ErrorIs(t, err, context.Canceled) require.Nil(t, ids) @@ -82,7 +85,7 @@ func TestSource_Reset(t *testing.T) { t.Run("success", func(t *testing.T) { ids, err := repoSource.Insert(ctx, - sourcesJobs(sourceId, destinationId, workspaceId, 10), + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 10), ) require.NoError(t, err) require.Len(t, ids, 10) @@ -125,3 +128,151 @@ func TestSource_Reset(t *testing.T) { require.ErrorIs(t, err, context.Canceled) }) } + +func TestSource_GetToProcess(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"key": "value"}`), 25), + ) + require.NoError(t, err) + require.Len(t, ids, 25) + + t.Run("less jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 25) + require.NoError(t, err) + require.Len(t, jobs, 25) + }) + t.Run("more jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 20) + require.NoError(t, err) + require.Len(t, jobs, 20) + }) + t.Run("equal jobs", func(t *testing.T) { + jobs, err := repoSource.GetToProcess(ctx, 50) + require.NoError(t, err) + require.Len(t, jobs, 25) + }) + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + jobs, err := repoSource.GetToProcess(ctx, 10) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, jobs) + }) +} + +func TestSource_GetByJobRunTaskRun(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + jobRun = "test-job-run" + taskRun = "test-task-run" + otherJobRun = "other-job-run" + otherTaskRun = "other-task-run" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("job is available", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), 1), + ) + require.NoError(t, err) + require.Len(t, ids, 1) + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.NoError(t, err) + require.Equal(t, job, &model.SourceJob{ + ID: 1, + SourceID: sourceId, + DestinationID: destinationId, + WorkspaceID: workspaceId, + TableName: "table0", + Status: model.SourceJobStatusWaiting, + Error: nil, + JobType: model.DeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 0, + }) + }) + t.Run("job is not available", func(t *testing.T) { + job, err := repoSource.GetByJobRunTaskRun(ctx, otherJobRun, otherTaskRun) + require.ErrorIs(t, err, model.ErrSourcesJobNotFound) + require.Nil(t, job) + }) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, job) + }) +} + +func TestSource_OnUpdateSuccess(t *testing.T) { + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) {}) + t.Run("job not found", func(t *testing.T) {}) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + err := repoSource.OnUpdateSuccess(ctx, int64(1)) + require.ErrorIs(t, err, context.Canceled) + }) +} + +func TestSource_OnUpdateFailure(t *testing.T) { + const ( + testError = "test-error" + ) + + db, ctx := setupDB(t), context.Background() + + now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) + repoSource := repo.NewSource(db, repo.WithNow(func() time.Time { + return now + })) + + t.Run("success", func(t *testing.T) {}) + t.Run("job not found", func(t *testing.T) {}) + t.Run("context cancelled", func(t *testing.T) { + ctx, cancel := context.WithCancel(ctx) + cancel() + + err := repoSource.OnUpdateFailure(ctx, int64(1), errors.New(testError), 1) + require.ErrorIs(t, err, context.Canceled) + }) +} diff --git a/warehouse/internal/repo/table_upload.go b/warehouse/internal/repo/table_upload.go index 80fcc820b2..e0fbd550d7 100644 --- a/warehouse/internal/repo/table_upload.go +++ b/warehouse/internal/repo/table_upload.go @@ -91,15 +91,7 @@ func (repo *TableUploads) Insert(ctx context.Context, uploadID int64, tableNames defer func() { _ = stmt.Close() }() for _, tableName := range tableNames { - _, err = stmt.ExecContext( - ctx, - uploadID, - tableName, - model.TableUploadWaiting, - "{}", - repo.now(), - repo.now(), - ) + _, err = stmt.ExecContext(ctx, uploadID, tableName, model.TableUploadWaiting, "{}", repo.now(), repo.now()) if err != nil { return fmt.Errorf(`stmt exec: %w`, err) } @@ -120,6 +112,7 @@ func (repo *TableUploads) GetByUploadID(ctx context.Context, uploadID int64) ([] if err != nil { return nil, fmt.Errorf("querying table uploads: %w", err) } + defer func() { _ = rows.Close() }() var tableUploads []model.TableUpload for rows.Next() { diff --git a/warehouse/slave/worker.go b/warehouse/slave/worker.go index 1f9865097f..0051f91d81 100644 --- a/warehouse/slave/worker.go +++ b/warehouse/slave/worker.go @@ -51,7 +51,7 @@ type uploadResult struct { UseRudderStorage bool } -type asyncJobRunResult struct { +type sourceJobRunResult struct { Result bool `json:"Result"` ID int64 `json:"Id"` } @@ -134,7 +134,7 @@ func (w *worker) start(ctx context.Context, notificationChan <-chan *notifier.Cl switch claimedJob.Job.Type { case notifier.JobTypeAsync: - w.processClaimedAsyncJob(ctx, claimedJob) + w.processClaimedSourceJob(ctx, claimedJob) default: w.processClaimedUploadJob(ctx, claimedJob) } @@ -424,7 +424,7 @@ func (w *worker) processStagingFile(ctx context.Context, job payload) ([]uploadR return uploadsResults, err } -func (w *worker) processClaimedAsyncJob(ctx context.Context, claimedJob *notifier.ClaimJob) { +func (w *worker) processClaimedSourceJob(ctx context.Context, claimedJob *notifier.ClaimJob) { handleErr := func(err error, claimedJob *notifier.ClaimJob) { w.log.Errorf("Error processing claim: %v", err) @@ -443,7 +443,7 @@ func (w *worker) processClaimedAsyncJob(ctx context.Context, claimedJob *notifie return } - jobResult, err := w.runAsyncJob(ctx, job) + jobResult, err := w.runSourceJob(ctx, job) if err != nil { handleErr(err, claimedJob) return @@ -460,13 +460,13 @@ func (w *worker) processClaimedAsyncJob(ctx context.Context, claimedJob *notifie }) } -func (w *worker) runAsyncJob(ctx context.Context, asyncjob model.SourceJob) (asyncJobRunResult, error) { - result := asyncJobRunResult{ - ID: asyncjob.ID, +func (w *worker) runSourceJob(ctx context.Context, sourceJob model.SourceJob) (sourceJobRunResult, error) { + result := sourceJobRunResult{ + ID: sourceJob.ID, Result: false, } - warehouse, err := w.destinationFromSlaveConnectionMap(asyncjob.DestinationID, asyncjob.SourceID) + warehouse, err := w.destinationFromSlaveConnectionMap(sourceJob.DestinationID, sourceJob.SourceID) if err != nil { return result, err } @@ -488,20 +488,20 @@ func (w *worker) runAsyncJob(ctx context.Context, asyncjob model.SourceJob) (asy defer integrationsManager.Cleanup(ctx) var metadata warehouseutils.DeleteByMetaData - if err = json.Unmarshal(asyncjob.Metadata, &metadata); err != nil { + if err = json.Unmarshal(sourceJob.Metadata, &metadata); err != nil { return result, err } - switch asyncjob.JobType { + switch sourceJob.JobType { case model.DeleteByJobRunID: - err = integrationsManager.DeleteBy(ctx, []string{asyncjob.TableName}, warehouseutils.DeleteByParams{ - SourceId: asyncjob.SourceID, + err = integrationsManager.DeleteBy(ctx, []string{sourceJob.TableName}, warehouseutils.DeleteByParams{ + SourceId: sourceJob.SourceID, TaskRunId: metadata.TaskRunId, JobRunId: metadata.JobRunId, StartTime: metadata.StartTime, }) default: - err = errors.New("invalid asyncJob type") + err = errors.New("invalid sourceJob type") } if err != nil { return result, err diff --git a/warehouse/slave/worker_test.go b/warehouse/slave/worker_test.go index dec421fe90..5b1969e045 100644 --- a/warehouse/slave/worker_test.go +++ b/warehouse/slave/worker_test.go @@ -604,13 +604,13 @@ func TestSlaveWorker(t *testing.T) { go func() { defer close(claimedJobDone) - slaveWorker.processClaimedAsyncJob(ctx, claim) + slaveWorker.processClaimedSourceJob(ctx, claim) }() response := <-subscribeCh require.NoError(t, response.Err) - var asyncResult asyncJobRunResult + var asyncResult sourceJobRunResult err = json.Unmarshal(response.Payload, &asyncResult) require.NoError(t, err) @@ -654,7 +654,7 @@ func TestSlaveWorker(t *testing.T) { sourceID: sourceID, destinationID: destinationID, jobType: "invalid_job_type", - expectedError: errors.New("invalid asyncJob type"), + expectedError: errors.New("invalid sourceJob type"), }, { name: "invalid parameters", @@ -710,7 +710,7 @@ func TestSlaveWorker(t *testing.T) { go func() { defer close(claimedJobDone) - slaveWorker.processClaimedAsyncJob(ctx, claim) + slaveWorker.processClaimedSourceJob(ctx, claim) }() response := <-subscribeCh diff --git a/warehouse/source/http.go b/warehouse/source/http.go index 56f5a3dbc9..6c742c43cd 100644 --- a/warehouse/source/http.go +++ b/warehouse/source/http.go @@ -43,13 +43,13 @@ func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { var payload insertJobRequest if err := json.NewDecoder(r.Body).Decode(&payload); err != nil { - a.logger.Warnw("invalid JSON in request body for inserting async jobs", lf.Error, err.Error()) + a.logger.Warnw("invalid JSON in request body for inserting source jobs", lf.Error, err.Error()) http.Error(w, ierrors.ErrInvalidJSONRequestBody.Error(), http.StatusBadRequest) return } if err := validatePayload(&payload); err != nil { - a.logger.Warnw("invalid payload for inserting async job", lf.Error, err.Error()) + a.logger.Warnw("invalid payload for inserting source job", lf.Error, err.Error()) http.Error(w, fmt.Sprintf("invalid payload: %s", err.Error()), http.StatusBadRequest) return } @@ -66,7 +66,7 @@ func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { http.Error(w, ierrors.ErrRequestCancelled.Error(), http.StatusBadRequest) return } - a.logger.Errorw("extracting tableNames for inserting async job", lf.Error, err.Error()) + a.logger.Errorw("extracting tableNames for inserting source job", lf.Error, err.Error()) http.Error(w, "can't extract tableNames", http.StatusInternalServerError) return } @@ -97,7 +97,7 @@ func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { JobType: string(notifier.JobTypeAsync), }) if err != nil { - a.logger.Errorw("marshalling metadata for inserting async job", lf.Error, err.Error()) + a.logger.Errorw("marshalling metadata for inserting source job", lf.Error, err.Error()) http.Error(w, "can't marshall metadata", http.StatusInternalServerError) return } @@ -128,7 +128,7 @@ func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { Err: nil, }) if err != nil { - a.logger.Errorw("marshalling response for inserting async job", lf.Error, err.Error()) + a.logger.Errorw("marshalling response for inserting source job", lf.Error, err.Error()) http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) return } @@ -149,7 +149,7 @@ func (a *Manager) StatusJobHandler(w http.ResponseWriter, r *http.Request) { WorkspaceID: queryParams.Get("workspace_id"), } if err := validatePayload(&payload); err != nil { - a.logger.Warnw("invalid payload for async job status", lf.Error, err.Error()) + a.logger.Warnw("invalid payload for source job status", lf.Error, err.Error()) http.Error(w, fmt.Sprintf("invalid request: %s", err.Error()), http.StatusBadRequest) return } @@ -160,8 +160,8 @@ func (a *Manager) StatusJobHandler(w http.ResponseWriter, r *http.Request) { http.Error(w, ierrors.ErrRequestCancelled.Error(), http.StatusBadRequest) return } - a.logger.Warnw("unable to get async job status", lf.Error, err.Error()) - http.Error(w, fmt.Sprintf("can't get async job status: %s", err.Error()), http.StatusBadRequest) + a.logger.Warnw("unable to get source job status", lf.Error, err.Error()) + http.Error(w, fmt.Sprintf("can't get source job status: %s", err.Error()), http.StatusBadRequest) return } @@ -182,7 +182,7 @@ func (a *Manager) StatusJobHandler(w http.ResponseWriter, r *http.Request) { resBody, err := json.Marshal(statusResponse) if err != nil { - a.logger.Errorw("marshalling response for async job status", lf.Error, err.Error()) + a.logger.Errorw("marshalling response for source job status", lf.Error, err.Error()) http.Error(w, ierrors.ErrMarshallResponse.Error(), http.StatusInternalServerError) return } diff --git a/warehouse/source/http_test.go b/warehouse/source/http_test.go index 70d497e015..97f8c00cc1 100644 --- a/warehouse/source/http_test.go +++ b/warehouse/source/http_test.go @@ -30,7 +30,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestAsyncJobHandlers(t *testing.T) { +func TestHTTPHandlers(t *testing.T) { const ( workspaceID = "test_workspace_id" sourceID = "test_source_id" @@ -199,18 +199,6 @@ func TestAsyncJobHandlers(t *testing.T) { }) t.Run("InsertJobHandler", func(t *testing.T) { - t.Run("Not enabled", func(t *testing.T) { - req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", nil) - resp := httptest.NewRecorder() - - jobsManager := New(config.New(), logger.NOP, db, n) - jobsManager.InsertJobHandler(resp, req) - require.Equal(t, http.StatusInternalServerError, resp.Code) - - b, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, "warehouse jobs api not initialized\n", string(b)) - }) t.Run("invalid payload", func(t *testing.T) { req := httptest.NewRequest(http.MethodPost, "/v1/warehouse/jobs", bytes.NewReader([]byte(`"Invalid payload"`))) resp := httptest.NewRecorder() @@ -262,18 +250,6 @@ func TestAsyncJobHandlers(t *testing.T) { }) t.Run("StatusJobHandler", func(t *testing.T) { - t.Run("Not enabled", func(t *testing.T) { - req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) - resp := httptest.NewRecorder() - - jobsManager := New(config.New(), logger.NOP, db, n) - jobsManager.StatusJobHandler(resp, req) - require.Equal(t, http.StatusInternalServerError, resp.Code) - - b, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, "warehouse jobs api not initialized\n", string(b)) - }) t.Run("invalid payload", func(t *testing.T) { req := httptest.NewRequest(http.MethodGet, "/v1/warehouse/jobs/status", nil) resp := httptest.NewRecorder() diff --git a/warehouse/source/source.go b/warehouse/source/source.go index 8d78ab2b60..dd9b46ca3d 100644 --- a/warehouse/source/source.go +++ b/warehouse/source/source.go @@ -37,7 +37,6 @@ type Manager struct { } } -// New Initializes AsyncJobWh structure with appropriate variabless func New( conf *config.Config, log logger.Logger, @@ -69,7 +68,7 @@ func (a *Manager) Run(ctx context.Context) error { /* startProcessing is the main runner that -1) Periodically queries the db for any pending async jobs +1) Periodically queries the db for any pending source jobs 2) Groups them together 3) Publishes them to the notifier 4) Spawns a subroutine that periodically checks for responses from Notifier/slave worker post trackBatch @@ -84,7 +83,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { pendingJobs, err := a.sourceRepo.GetToProcess(ctx, a.config.maxBatchSizeToProcess) if err != nil { - return fmt.Errorf("unable to get pending async jobs with error %s", err.Error()) + return fmt.Errorf("unable to get pending source jobs with error %s", err.Error()) } if len(pendingJobs) == 0 { continue @@ -94,7 +93,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { for _, job := range pendingJobs { message, err := json.Marshal(job) if err != nil { - return fmt.Errorf("unable to marshal async job payload with error %s", err.Error()) + return fmt.Errorf("unable to marshal source job payload with error %s", err.Error()) } notifierClaims = append(notifierClaims, message) } @@ -105,7 +104,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { Priority: 100, }) if err != nil { - return fmt.Errorf("unable to publish async jobs to notifier with error %s", err.Error()) + return fmt.Errorf("unable to publish source jobs to notifier with error %s", err.Error()) } pendingJobsMap := make(map[int64]model.SourceJob) @@ -127,7 +126,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { a.config.maxAttemptsPerJob, ) if err != nil { - return fmt.Errorf("unable to update async job with error %s", err.Error()) + return fmt.Errorf("unable to update source job with error %s", err.Error()) } } continue @@ -141,7 +140,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { a.config.maxAttemptsPerJob, ) if err != nil { - return fmt.Errorf("unable to update async job with error %s", err.Error()) + return fmt.Errorf("unable to update source job with error %s", err.Error()) } } continue @@ -155,7 +154,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { } if pj, ok := pendingJobsMap[response.Id]; ok { - pj.Status = job.Status + pj.Status = string(job.Status) pj.Error = job.Error } } @@ -169,7 +168,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { a.config.maxAttemptsPerJob, ) if err != nil { - return fmt.Errorf("unable to update async job with error %s", err.Error()) + return fmt.Errorf("unable to update source job with error %s", err.Error()) } continue } @@ -178,7 +177,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { job.ID, ) if err != nil { - return fmt.Errorf("unable to update async job with error %s", err.Error()) + return fmt.Errorf("unable to update source job with error %s", err.Error()) } } case <-time.After(a.config.processingTimeout): @@ -190,7 +189,7 @@ func (a *Manager) startProcessing(ctx context.Context) error { a.config.maxAttemptsPerJob, ) if err != nil { - return fmt.Errorf("unable to update async job with error %s", err.Error()) + return fmt.Errorf("unable to update source job with error %s", err.Error()) } } } From 730fc03b21c6505f8bb6a3ac7b962bd5188d6be0 Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Sun, 15 Oct 2023 03:12:58 +0530 Subject: [PATCH 4/7] some more changes --- warehouse/internal/repo/source.go | 18 ++-- warehouse/internal/repo/source_test.go | 110 +++++++++++++++++++++++-- warehouse/slave/worker.go | 2 +- warehouse/source/http.go | 3 +- warehouse/source/http_test.go | 18 ++-- warehouse/source/source.go | 42 ++++------ 6 files changed, 140 insertions(+), 53 deletions(-) diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go index 8991418fc5..5fddfc24a8 100644 --- a/warehouse/internal/repo/source.go +++ b/warehouse/internal/repo/source.go @@ -258,7 +258,7 @@ func (repo *Source) OnUpdateSuccess( return fmt.Errorf("rows affected: %w", err) } if rowsAffected == 0 { - return fmt.Errorf("no rows affected") + return model.ErrSourcesJobNotFound } return nil @@ -274,17 +274,19 @@ func (repo *Source) OnUpdateFailure( UPDATE `+sourceJobTableName+` SET - status =( - CASE WHEN attempt > $1 THEN `+model.SourceJobStatusAborted+` - ELSE `+model.SourceJobStatusFailed+`) END + status = ( + CASE WHEN attempt > $1 THEN $2 + ELSE $3 END ), attempt = attempt + 1, - updated_at = $2, - error = $3 + updated_at = $4, + error = $5 WHERE - id = $3; + id = $6; `, maxAttempt, + model.SourceJobStatusAborted, + model.SourceJobStatusFailed, repo.now(), error.Error(), id, @@ -297,7 +299,7 @@ func (repo *Source) OnUpdateFailure( return fmt.Errorf("rows affected: %w", err) } if rowsAffected == 0 { - return fmt.Errorf("no rows affected") + return model.ErrSourcesJobNotFound } return nil diff --git a/warehouse/internal/repo/source_test.go b/warehouse/internal/repo/source_test.go index 8af6f98ef3..d796005be1 100644 --- a/warehouse/internal/repo/source_test.go +++ b/warehouse/internal/repo/source_test.go @@ -16,6 +16,7 @@ import ( "github.com/rudderlabs/rudder-server/warehouse/internal/repo" ) +//nolint:unparam func sourcesJobs( sourceID, destinationID, workspaceID string, jobType string, metadata json.RawMessage, @@ -236,6 +237,14 @@ func TestSource_GetByJobRunTaskRun(t *testing.T) { } func TestSource_OnUpdateSuccess(t *testing.T) { + const ( + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + jobRun = "test-job-run" + taskRun = "test-task-run" + ) + db, ctx := setupDB(t), context.Background() now := time.Date(2023, 1, 1, 0, 0, 0, 0, time.UTC) @@ -243,8 +252,39 @@ func TestSource_OnUpdateSuccess(t *testing.T) { return now })) - t.Run("success", func(t *testing.T) {}) - t.Run("job not found", func(t *testing.T) {}) + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), 1), + ) + require.NoError(t, err) + require.Len(t, ids, 1) + + err = repoSource.OnUpdateSuccess(ctx, int64(1)) + require.NoError(t, err) + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.NoError(t, err) + + require.Equal(t, job, &model.SourceJob{ + ID: 1, + SourceID: sourceId, + DestinationID: destinationId, + WorkspaceID: workspaceId, + TableName: "table0", + Status: model.SourceJobStatusSucceeded, + Error: nil, + JobType: model.DeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 0, + }) + }) + t.Run("job not found", func(t *testing.T) { + err := repoSource.OnUpdateSuccess(ctx, int64(-1)) + require.ErrorIs(t, err, model.ErrSourcesJobNotFound) + }) t.Run("context cancelled", func(t *testing.T) { ctx, cancel := context.WithCancel(ctx) cancel() @@ -256,7 +296,12 @@ func TestSource_OnUpdateSuccess(t *testing.T) { func TestSource_OnUpdateFailure(t *testing.T) { const ( - testError = "test-error" + sourceId = "test_source_id" + destinationId = "test_destination_id" + workspaceId = "test_workspace_id" + jobRun = "test-job-run" + taskRun = "test-task-run" + testError = "test-error" ) db, ctx := setupDB(t), context.Background() @@ -266,8 +311,63 @@ func TestSource_OnUpdateFailure(t *testing.T) { return now })) - t.Run("success", func(t *testing.T) {}) - t.Run("job not found", func(t *testing.T) {}) + t.Run("success", func(t *testing.T) { + ids, err := repoSource.Insert( + ctx, + sourcesJobs(sourceId, destinationId, workspaceId, model.DeleteByJobRunID, json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), 1), + ) + require.NoError(t, err) + require.Len(t, ids, 1) + + t.Run("not crossed max attempt", func(t *testing.T) { + err = repoSource.OnUpdateFailure(ctx, int64(1), errors.New(testError), 1) + require.NoError(t, err) + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.NoError(t, err) + + require.Equal(t, job, &model.SourceJob{ + ID: 1, + SourceID: sourceId, + DestinationID: destinationId, + WorkspaceID: workspaceId, + TableName: "table0", + Status: model.Failed, + Error: errors.New(testError), + JobType: model.DeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 1, + }) + }) + t.Run("crossed max attempt", func(t *testing.T) { + err = repoSource.OnUpdateFailure(ctx, int64(1), errors.New(testError), -1) + require.NoError(t, err) + + job, err := repoSource.GetByJobRunTaskRun(ctx, jobRun, taskRun) + require.NoError(t, err) + + require.Equal(t, job, &model.SourceJob{ + ID: 1, + SourceID: sourceId, + DestinationID: destinationId, + WorkspaceID: workspaceId, + TableName: "table0", + Status: model.Aborted, + Error: errors.New(testError), + JobType: model.DeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 2, + }) + }) + }) + t.Run("job not found", func(t *testing.T) { + err := repoSource.OnUpdateFailure(ctx, int64(-1), errors.New(testError), 1) + require.ErrorIs(t, err, model.ErrSourcesJobNotFound) + }) t.Run("context cancelled", func(t *testing.T) { ctx, cancel := context.WithCancel(ctx) cancel() diff --git a/warehouse/slave/worker.go b/warehouse/slave/worker.go index 0051f91d81..e3de17f1d0 100644 --- a/warehouse/slave/worker.go +++ b/warehouse/slave/worker.go @@ -481,7 +481,7 @@ func (w *worker) runSourceJob(ctx context.Context, sourceJob model.SourceJob) (s warehouse.Destination.ID, )) - err = integrationsManager.Setup(ctx, warehouse, &source.SourceUploader{}) + err = integrationsManager.Setup(ctx, warehouse, &source.Uploader{}) if err != nil { return result, err } diff --git a/warehouse/source/http.go b/warehouse/source/http.go index 6c742c43cd..e890b9e995 100644 --- a/warehouse/source/http.go +++ b/warehouse/source/http.go @@ -24,6 +24,7 @@ type insertJobRequest struct { StartTime string `json:"start_time"` JobRunID string `json:"job_run_id"` TaskRunID string `json:"task_run_id"` + JobType string `json:"async_job_type"` WorkspaceID string `json:"workspace_id"` } @@ -108,7 +109,7 @@ func (a *Manager) InsertJobHandler(w http.ResponseWriter, r *http.Request) { DestinationID: payload.DestinationID, WorkspaceID: payload.WorkspaceID, TableName: item, - JobType: model.DeleteByJobRunID, + JobType: payload.JobType, Metadata: metadataJson, } }) diff --git a/warehouse/source/http_test.go b/warehouse/source/http_test.go index 97f8c00cc1..ac999344d7 100644 --- a/warehouse/source/http_test.go +++ b/warehouse/source/http_test.go @@ -229,15 +229,13 @@ func TestHTTPHandlers(t *testing.T) { "source_id": "test_source_id", "destination_id": "test_destination_id", "job_run_id": "test_source_job_run_id", - "task_run_id": "test_source_task_run_id" + "task_run_id": "test_source_task_run_id", + "async_job_type": "deletebyjobrunid" } `))) resp := httptest.NewRecorder() - jobsManager := Manager{ - logger: logger.NOP, - notifier: n, - } + jobsManager := New(config.New(), logger.NOP, db, n) jobsManager.InsertJobHandler(resp, req) require.Equal(t, http.StatusOK, resp.Code) @@ -263,12 +261,6 @@ func TestHTTPHandlers(t *testing.T) { require.Equal(t, "invalid request: source_id is required\n", string(b)) }) t.Run("success", func(t *testing.T) { - _, err := db.ExecContext(ctx, ` - INSERT INTO `+whutils.WarehouseAsyncJobTable+` (source_id, destination_id, status, created_at, updated_at, tablename, error, async_job_type, metadata, workspace_id) - VALUES ('test_source_id', 'test_destination_id', 'aborted', NOW(), NOW(), 'test_table_name', 'test_error', 'deletebyjobrunid', '{"job_run_id": "test_source_job_run_id", "task_run_id": "test_source_task_run_id"}', 'test_workspace_id') - `) - require.NoError(t, err) - qp := url.Values{} qp.Add("task_run_id", sourceTaskRunID) qp.Add("job_run_id", sourceJobRunID) @@ -286,8 +278,8 @@ func TestHTTPHandlers(t *testing.T) { var statusResponse jobStatusResponse err = json.NewDecoder(resp.Body).Decode(&statusResponse) require.NoError(t, err) - require.Equal(t, statusResponse.Status, "aborted") - require.Equal(t, statusResponse.Err, "test_error") + require.Equal(t, statusResponse.Status, model.SourceJobStatusExecuting) + require.Empty(t, statusResponse.Err) }) }) } diff --git a/warehouse/source/source.go b/warehouse/source/source.go index dd9b46ca3d..c1651301c5 100644 --- a/warehouse/source/source.go +++ b/warehouse/source/source.go @@ -196,32 +196,24 @@ func (a *Manager) startProcessing(ctx context.Context) error { } } -type SourceUploader struct{} - -func (*SourceUploader) IsWarehouseSchemaEmpty() bool { return true } -func (*SourceUploader) UpdateLocalSchema(context.Context, model.Schema) error { return nil } -func (*SourceUploader) GetTableSchemaInUpload(string) model.TableSchema { return model.TableSchema{} } -func (*SourceUploader) GetTableSchemaInWarehouse(string) model.TableSchema { - return model.TableSchema{} -} -func (*SourceUploader) ShouldOnDedupUseNewRecord() bool { return false } -func (*SourceUploader) UseRudderStorage() bool { return false } -func (*SourceUploader) CanAppend() bool { return false } -func (*SourceUploader) GetLoadFileGenStartTIme() time.Time { return time.Time{} } -func (*SourceUploader) GetLoadFileType() string { return "" } -func (*SourceUploader) GetFirstLastEvent() (time.Time, time.Time) { return time.Now(), time.Now() } -func (*SourceUploader) GetLocalSchema(context.Context) (model.Schema, error) { - return model.Schema{}, nil -} - -func (*SourceUploader) GetSingleLoadFile(context.Context, string) (warehouseutils.LoadFile, error) { - return warehouseutils.LoadFile{}, nil -} - -func (*SourceUploader) GetLoadFilesMetadata(context.Context, warehouseutils.GetLoadFilesOptions) []warehouseutils.LoadFile { +type Uploader struct{} + +func (*Uploader) IsWarehouseSchemaEmpty() bool { return true } +func (*Uploader) UpdateLocalSchema(context.Context, model.Schema) error { return nil } +func (*Uploader) GetTableSchemaInUpload(string) model.TableSchema { return model.TableSchema{} } +func (*Uploader) ShouldOnDedupUseNewRecord() bool { return false } +func (*Uploader) UseRudderStorage() bool { return false } +func (*Uploader) CanAppend() bool { return false } +func (*Uploader) GetLoadFileGenStartTIme() time.Time { return time.Time{} } +func (*Uploader) GetLoadFileType() string { return "" } +func (*Uploader) GetFirstLastEvent() (time.Time, time.Time) { return time.Now(), time.Now() } +func (*Uploader) GetLocalSchema(context.Context) (model.Schema, error) { return model.Schema{}, nil } +func (*Uploader) GetTableSchemaInWarehouse(string) model.TableSchema { return model.TableSchema{} } +func (*Uploader) GetSampleLoadFileLocation(context.Context, string) (string, error) { return "", nil } +func (*Uploader) GetLoadFilesMetadata(context.Context, warehouseutils.GetLoadFilesOptions) []warehouseutils.LoadFile { return []warehouseutils.LoadFile{} } -func (*SourceUploader) GetSampleLoadFileLocation(context.Context, string) (string, error) { - return "", nil +func (*Uploader) GetSingleLoadFile(context.Context, string) (warehouseutils.LoadFile, error) { + return warehouseutils.LoadFile{}, nil } From 598fab52958925dd37bd07e4ca4dc90edf382e22 Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Tue, 31 Oct 2023 01:59:43 +0530 Subject: [PATCH 5/7] chore: review comments --- warehouse/internal/repo/source.go | 69 +++++++++++-------------- warehouse/internal/repo/source_test.go | 28 +++++----- warehouse/internal/repo/table_upload.go | 10 ++-- warehouse/slave/worker.go | 40 +++++--------- warehouse/slave/worker_test.go | 27 +++++----- warehouse/source/http_test.go | 6 +-- warehouse/source/source.go | 46 ++++++++++------- warehouse/source/source_test.go | 1 - warehouse/source/types.go | 15 +++++- 9 files changed, 119 insertions(+), 123 deletions(-) diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go index b6beda144d..ebcc882880 100644 --- a/warehouse/internal/repo/source.go +++ b/warehouse/internal/repo/source.go @@ -80,10 +80,10 @@ func (s *Source) Insert(ctx context.Context, sourceJobs []model.SourceJob) ([]in sourceJob.SourceID, sourceJob.DestinationID, sourceJob.TableName, - model.SourceJobStatusWaiting, + model.SourceJobStatusWaiting.String(), s.now(), s.now(), - sourceJob.JobType, + sourceJob.JobType.String(), sourceJob.WorkspaceID, sourceJob.Metadata, ).Scan(&id) @@ -109,9 +109,9 @@ func (s *Source) Reset(ctx context.Context) error { WHERE status = $2 OR status = $3; `, - model.SourceJobStatusWaiting, - model.SourceJobStatusExecuting, - model.SourceJobStatusFailed, + model.SourceJobStatusWaiting.String(), + model.SourceJobStatusExecuting.String(), + model.SourceJobStatusFailed.String(), ) if err != nil { return fmt.Errorf("executing: %w", err) @@ -129,8 +129,8 @@ func (s *Source) GetToProcess(ctx context.Context, limit int64) ([]model.SourceJ status = $1 OR status = $2 LIMIT $3; `, - model.SourceJobStatusWaiting, - model.SourceJobStatusFailed, + model.SourceJobStatusWaiting.String(), + model.SourceJobStatusFailed.String(), limit, ) if err != nil { @@ -140,7 +140,7 @@ func (s *Source) GetToProcess(ctx context.Context, limit int64) ([]model.SourceJ sourceJobs, err := scanSourceJobs(rows) if err != nil { - return nil, fmt.Errorf("scanning: %w", err) + return nil, fmt.Errorf("scanning source jobs: %w", err) } return sourceJobs, nil } @@ -151,7 +151,7 @@ func scanSourceJobs(rows *sqlmw.Rows) ([]model.SourceJob, error) { var sourceJob model.SourceJob err := scanSourceJob(rows.Scan, &sourceJob) if err != nil { - return nil, err + return nil, fmt.Errorf("scanning source job: %w", err) } sourceJobs = append(sourceJobs, sourceJob) } @@ -163,49 +163,45 @@ func scanSourceJobs(rows *sqlmw.Rows) ([]model.SourceJob, error) { func scanSourceJob(scan scanFn, sourceJob *model.SourceJob) error { var errorRaw sql.NullString - var jobTypeRaw sql.NullString + var jobType, status string if err := scan( &sourceJob.ID, &sourceJob.SourceID, &sourceJob.DestinationID, - &sourceJob.Status, + &status, &sourceJob.CreatedAt, &sourceJob.UpdatedAt, &sourceJob.TableName, &errorRaw, - &jobTypeRaw, + &jobType, &sourceJob.Metadata, &sourceJob.Attempts, &sourceJob.WorkspaceID, ); err != nil { - return fmt.Errorf("scanning: %w", err) + return fmt.Errorf("scanning row: %w", err) } - if errorRaw.Valid { - sourceJob.Error = errors.New(errorRaw.String) + sourceJobStatus, err := model.FromSourceJobStatus(status) + if err != nil { + return fmt.Errorf("getting sourceJobStatus %w", err) } - if jobTypeRaw.Valid { - jobType, err := model.FromSourceJobType(jobTypeRaw.String) - if err != nil { - return fmt.Errorf("scanning: %w", err) - } - sourceJob.JobType = jobType - } else { - return fmt.Errorf("scanning: job type is null") + sourceJobType, err := model.FromSourceJobType(jobType) + if err != nil { + return fmt.Errorf("getting sourceJobType: %w", err) + } + if errorRaw.Valid && errorRaw.String != "" { + sourceJob.Error = errors.New(errorRaw.String) } + sourceJob.Status = sourceJobStatus + sourceJob.JobType = sourceJobType sourceJob.CreatedAt = sourceJob.CreatedAt.UTC() sourceJob.UpdatedAt = sourceJob.UpdatedAt.UTC() - return nil } -func (s *Source) GetByJobRunTaskRun( - ctx context.Context, - jobRunID string, - taskRunID string, -) (*model.SourceJob, error) { +func (s *Source) GetByJobRunTaskRun(ctx context.Context, jobRunID, taskRunID string) (*model.SourceJob, error) { row := s.db.QueryRowContext(ctx, ` SELECT `+sourceJobColumns+` @@ -226,7 +222,7 @@ func (s *Source) GetByJobRunTaskRun( return nil, model.ErrSourcesJobNotFound } if err != nil { - return nil, err + return nil, fmt.Errorf("scanning source job: %w", err) } return &sourceJob, nil } @@ -246,7 +242,7 @@ func (s *Source) OnUpdateSuccess(ctx context.Context, id int64) error { id, ) if err != nil { - return fmt.Errorf("on update success: %w", err) + return fmt.Errorf("executing: %w", err) } rowsAffected, err := r.RowsAffected() if err != nil { @@ -258,12 +254,7 @@ func (s *Source) OnUpdateSuccess(ctx context.Context, id int64) error { return nil } -func (s *Source) OnUpdateFailure( - ctx context.Context, - id int64, - error error, - maxAttempt int, -) error { +func (s *Source) OnUpdateFailure(ctx context.Context, id int64, error error, maxAttempt int) error { r, err := s.db.ExecContext(ctx, ` UPDATE `+sourceJobTableName+` @@ -286,7 +277,7 @@ func (s *Source) OnUpdateFailure( id, ) if err != nil { - return fmt.Errorf("on update failed: %w", err) + return fmt.Errorf("executing: %w", err) } rowsAffected, err := r.RowsAffected() if err != nil { @@ -313,7 +304,7 @@ func (s *Source) MarkExecuting(ctx context.Context, ids []int64) error { pq.Array(ids), ) if err != nil { - return fmt.Errorf("query: %w", err) + return fmt.Errorf("executing: %w", err) } return nil } diff --git a/warehouse/internal/repo/source_test.go b/warehouse/internal/repo/source_test.go index 1ca8ed1675..4115cea621 100644 --- a/warehouse/internal/repo/source_test.go +++ b/warehouse/internal/repo/source_test.go @@ -379,13 +379,13 @@ func TestSource_OnUpdateFailure(t *testing.T) { DestinationID: destinationId, WorkspaceID: workspaceId, TableName: "table-0", - // Status: model.Failed, - Error: errors.New(testError), - JobType: model.SourceJobTypeDeleteByJobRunID, - Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), - CreatedAt: now.UTC(), - UpdatedAt: now.UTC(), - Attempts: 1, + Status: model.SourceJobStatusFailed, + Error: errors.New(testError), + JobType: model.SourceJobTypeDeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 1, }) }) t.Run("crossed max attempt", func(t *testing.T) { @@ -401,13 +401,13 @@ func TestSource_OnUpdateFailure(t *testing.T) { DestinationID: destinationId, WorkspaceID: workspaceId, TableName: "table-0", - // Status: model.Aborted, - Error: errors.New(testError), - JobType: model.SourceJobTypeDeleteByJobRunID, - Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), - CreatedAt: now.UTC(), - UpdatedAt: now.UTC(), - Attempts: 2, + Status: model.SourceJobStatusAborted, + Error: errors.New(testError), + JobType: model.SourceJobTypeDeleteByJobRunID, + Metadata: json.RawMessage(`{"job_run_id": "test-job-run", "task_run_id": "test-task-run"}`), + CreatedAt: now.UTC(), + UpdatedAt: now.UTC(), + Attempts: 2, }) }) }) diff --git a/warehouse/internal/repo/table_upload.go b/warehouse/internal/repo/table_upload.go index 60536b2c96..83087a31b3 100644 --- a/warehouse/internal/repo/table_upload.go +++ b/warehouse/internal/repo/table_upload.go @@ -116,7 +116,7 @@ func (repo *TableUploads) GetByUploadID(ctx context.Context, uploadID int64) ([] tableUploads, err := scanTableUploads(rows) if err != nil { - return nil, fmt.Errorf("parsing rows: %w", err) + return nil, fmt.Errorf("scanning table uploads: %w", err) } return tableUploads, nil } @@ -137,7 +137,7 @@ func (repo *TableUploads) GetByUploadIDAndTableName(ctx context.Context, uploadI return tableUpload, fmt.Errorf("no table upload found with uploadID: %d, tableName: %s", uploadID, tableName) } if err != nil { - return tableUpload, fmt.Errorf("parsing rows: %w", err) + return tableUpload, fmt.Errorf("scanning table upload: %w", err) } return tableUpload, err } @@ -148,7 +148,7 @@ func scanTableUploads(rows *sqlmiddleware.Rows) ([]model.TableUpload, error) { var tableUpload model.TableUpload err := scanTableUpload(rows.Scan, &tableUpload) if err != nil { - return nil, fmt.Errorf("parsing rows: %w", err) + return nil, fmt.Errorf("scanning table upload: %w", err) } tableUploads = append(tableUploads, tableUpload) } @@ -437,13 +437,13 @@ func (repo *TableUploads) GetByJobRunTaskRun( taskRunID, ) if err != nil { - return nil, fmt.Errorf("getting table uploads: %w", err) + return nil, fmt.Errorf("querying: %w", err) } defer func() { _ = rows.Close() }() tableUploads, err := scanTableUploads(rows) if err != nil { - return nil, fmt.Errorf("parsing rows: %w", err) + return nil, fmt.Errorf("scanning table uploads: %w", err) } return tableUploads, nil } diff --git a/warehouse/slave/worker.go b/warehouse/slave/worker.go index f1bc77d914..e5b86d5f51 100644 --- a/warehouse/slave/worker.go +++ b/warehouse/slave/worker.go @@ -51,11 +51,6 @@ type uploadResult struct { UseRudderStorage bool } -type sourceJobRunResult struct { - Result bool `json:"Result"` - ID int64 `json:"Id"` -} - type worker struct { conf *config.Config log logger.Logger @@ -434,7 +429,7 @@ func (w *worker) processClaimedSourceJob(ctx context.Context, claimedJob *notifi } var ( - job model.SourceJob + job source.NotifierRequest err error ) @@ -443,13 +438,15 @@ func (w *worker) processClaimedSourceJob(ctx context.Context, claimedJob *notifi return } - jobResult, err := w.runSourceJob(ctx, job) + err = w.runSourceJob(ctx, job) if err != nil { handleErr(err, claimedJob) return } - jobResultJSON, err := json.Marshal(jobResult) + jobResultJSON, err := json.Marshal(source.NotifierResponse{ + ID: job.ID, + }) if err != nil { handleErr(err, claimedJob) return @@ -460,20 +457,15 @@ func (w *worker) processClaimedSourceJob(ctx context.Context, claimedJob *notifi }) } -func (w *worker) runSourceJob(ctx context.Context, sourceJob model.SourceJob) (sourceJobRunResult, error) { - result := sourceJobRunResult{ - ID: sourceJob.ID, - Result: false, - } - +func (w *worker) runSourceJob(ctx context.Context, sourceJob source.NotifierRequest) error { warehouse, err := w.destinationFromSlaveConnectionMap(sourceJob.DestinationID, sourceJob.SourceID) if err != nil { - return result, err + return fmt.Errorf("getting warehouse: %w", err) } integrationsManager, err := manager.NewWarehouseOperations(warehouse.Destination.DestinationDefinition.Name, w.conf, w.log, w.statsFactory) if err != nil { - return result, err + return fmt.Errorf("getting integrations manager: %w", err) } integrationsManager.SetConnectionTimeout(warehouseutils.GetConnectionTimeout( @@ -483,17 +475,17 @@ func (w *worker) runSourceJob(ctx context.Context, sourceJob model.SourceJob) (s err = integrationsManager.Setup(ctx, warehouse, &source.Uploader{}) if err != nil { - return result, err + return fmt.Errorf("setting up integrations manager: %w", err) } defer integrationsManager.Cleanup(ctx) var metadata warehouseutils.DeleteByMetaData - if err = json.Unmarshal(sourceJob.Metadata, &metadata); err != nil { - return result, err + if err = json.Unmarshal(sourceJob.MetaData, &metadata); err != nil { + return fmt.Errorf("unmarshalling metadata: %w", err) } switch sourceJob.JobType { - case model.SourceJobTypeDeleteByJobRunID: + case model.SourceJobTypeDeleteByJobRunID.String(): err = integrationsManager.DeleteBy(ctx, []string{sourceJob.TableName}, warehouseutils.DeleteByParams{ SourceId: sourceJob.SourceID, TaskRunId: metadata.TaskRunId, @@ -503,13 +495,7 @@ func (w *worker) runSourceJob(ctx context.Context, sourceJob model.SourceJob) (s default: err = errors.New("invalid sourceJob type") } - if err != nil { - return result, err - } - - result.Result = true - - return result, nil + return err } func (w *worker) destinationFromSlaveConnectionMap(destinationId, sourceId string) (model.Warehouse, error) { diff --git a/warehouse/slave/worker_test.go b/warehouse/slave/worker_test.go index 97aabf17c0..7e46e12fdf 100644 --- a/warehouse/slave/worker_test.go +++ b/warehouse/slave/worker_test.go @@ -9,6 +9,8 @@ import ( "os" "testing" + "github.com/rudderlabs/rudder-server/warehouse/source" + "github.com/rudderlabs/rudder-go-kit/stats/memstats" "github.com/rudderlabs/rudder-server/warehouse/bcm" @@ -576,14 +578,14 @@ func TestSlaveWorker(t *testing.T) { workerIdx, ) - p := model.SourceJob{ + p := source.NotifierRequest{ ID: 1, SourceID: sourceID, DestinationID: destinationID, TableName: "test_table_name", WorkspaceID: workspaceID, - JobType: model.SourceJobTypeDeleteByJobRunID, - Metadata: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), + JobType: model.SourceJobTypeDeleteByJobRunID.String(), + MetaData: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), } payloadJson, err := json.Marshal(p) @@ -610,12 +612,11 @@ func TestSlaveWorker(t *testing.T) { response := <-subscribeCh require.NoError(t, response.Err) - var asyncResult sourceJobRunResult - err = json.Unmarshal(response.Payload, &asyncResult) + var notifierResponse source.NotifierResponse + err = json.Unmarshal(response.Payload, ¬ifierResponse) require.NoError(t, err) - require.Equal(t, int64(1), asyncResult.ID) - require.True(t, asyncResult.Result) + require.Equal(t, int64(1), notifierResponse.ID) <-claimedJobDone }) @@ -652,21 +653,21 @@ func TestSlaveWorker(t *testing.T) { { name: "invalid parameters", jobType: model.SourceJobTypeDeleteByJobRunID, - expectedError: errors.New("invalid Parameters"), + expectedError: errors.New("getting warehouse: invalid Parameters"), }, { name: "invalid source id", sourceID: "invalid_source_id", destinationID: destinationID, jobType: model.SourceJobTypeDeleteByJobRunID, - expectedError: errors.New("invalid Source Id"), + expectedError: errors.New("getting warehouse: invalid Source Id"), }, { name: "invalid destination id", sourceID: sourceID, destinationID: "invalid_destination_id", jobType: model.SourceJobTypeDeleteByJobRunID, - expectedError: errors.New("invalid Destination Id"), + expectedError: errors.New("getting warehouse: invalid Destination Id"), }, } @@ -674,14 +675,14 @@ func TestSlaveWorker(t *testing.T) { tc := tc t.Run(tc.name, func(t *testing.T) { - p := model.SourceJob{ + p := source.NotifierRequest{ ID: 1, SourceID: tc.sourceID, DestinationID: tc.destinationID, TableName: "test_table_name", WorkspaceID: workspaceID, - JobType: tc.jobType, - Metadata: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), + JobType: tc.jobType.String(), + MetaData: []byte(`{"job_run_id": "1", "task_run_id": "1", "start_time": "2020-01-01T00:00:00Z"}`), } payloadJson, err := json.Marshal(p) diff --git a/warehouse/source/http_test.go b/warehouse/source/http_test.go index ddf30a4b25..d01bb48e8b 100644 --- a/warehouse/source/http_test.go +++ b/warehouse/source/http_test.go @@ -311,7 +311,7 @@ func TestManager_StatusJobHandler(t *testing.T) { var statusResponse jobStatusResponse err = json.NewDecoder(resp.Body).Decode(&statusResponse) require.NoError(t, err) - require.Equal(t, statusResponse.Status, model.SourceJobStatusWaiting) + require.Equal(t, statusResponse.Status, model.SourceJobStatusWaiting.String()) require.Empty(t, statusResponse.Err) }) t.Run("status aborted", func(t *testing.T) { @@ -339,7 +339,7 @@ func TestManager_StatusJobHandler(t *testing.T) { var statusResponse jobStatusResponse err = json.NewDecoder(resp.Body).Decode(&statusResponse) require.NoError(t, err) - require.Equal(t, statusResponse.Status, model.SourceJobStatusAborted) + require.Equal(t, statusResponse.Status, model.SourceJobStatusAborted.String()) require.Equal(t, statusResponse.Err, errors.New("test error").Error()) }) t.Run("job not found", func(t *testing.T) { @@ -355,7 +355,7 @@ func TestManager_StatusJobHandler(t *testing.T) { sourceManager := New(config.New(), logger.NOP, db, &mockPublisher{}) sourceManager.StatusJobHandler(resp, req) - require.Equal(t, http.StatusBadRequest, resp.Code) + require.Equal(t, http.StatusNotFound, resp.Code) b, err := io.ReadAll(resp.Body) require.NoError(t, err) diff --git a/warehouse/source/source.go b/warehouse/source/source.go index 69449a98ee..3723105c0c 100644 --- a/warehouse/source/source.go +++ b/warehouse/source/source.go @@ -59,17 +59,6 @@ func New(conf *config.Config, log logger.Logger, db *sqlmw.DB, publisher publish return m } -func (m *Manager) Run(ctx context.Context) error { - if err := m.sourceRepo.Reset(ctx); err != nil { - return fmt.Errorf("resetting source jobs with error %w", err) - } - - if err := m.process(ctx); err != nil && !errors.Is(err, context.Canceled) { - return fmt.Errorf("processing source jobs with error %w", err) - } - return nil -} - func (m *Manager) InsertJobs(ctx context.Context, payload insertJobRequest) ([]int64, error) { var jobType model.SourceJobType switch payload.JobType { @@ -115,14 +104,14 @@ func (m *Manager) InsertJobs(ctx context.Context, payload insertJobRequest) ([]i JobType: string(notifier.JobTypeAsync), }) if err != nil { - return nil, err + return nil, fmt.Errorf("marshalling metadata: %w", err) } - jobIds, err := m.sourceRepo.Insert(ctx, lo.Map(tableNames, func(item string, index int) model.SourceJob { + jobIds, err := m.sourceRepo.Insert(ctx, lo.Map(tableNames, func(tableName string, _ int) model.SourceJob { return model.SourceJob{ SourceID: payload.SourceID, DestinationID: payload.DestinationID, WorkspaceID: payload.WorkspaceID, - TableName: item, + TableName: tableName, JobType: jobType, Metadata: metadataJson, } @@ -133,6 +122,17 @@ func (m *Manager) InsertJobs(ctx context.Context, payload insertJobRequest) ([]i return jobIds, nil } +func (m *Manager) Run(ctx context.Context) error { + if err := m.sourceRepo.Reset(ctx); err != nil { + return fmt.Errorf("resetting source jobs with error %w", err) + } + + if err := m.process(ctx); err != nil && !errors.Is(err, context.Canceled) { + return fmt.Errorf("processing source jobs with error %w", err) + } + return nil +} + func (m *Manager) process(ctx context.Context) error { m.logger.Infow("starting source jobs processing") @@ -166,7 +166,15 @@ func (m *Manager) process(ctx context.Context) error { func (m *Manager) processPendingJobs(ctx context.Context, pendingJobs []model.SourceJob) error { claims := make([]json.RawMessage, 0, len(pendingJobs)) for _, job := range pendingJobs { - message, err := json.Marshal(job) + message, err := json.Marshal(NotifierRequest{ + ID: job.ID, + SourceID: job.SourceID, + DestinationID: job.DestinationID, + WorkspaceID: job.WorkspaceID, + TableName: job.TableName, + JobType: job.JobType.String(), + MetaData: job.Metadata, + }) if err != nil { return fmt.Errorf("marshalling source job %d: %w", job.ID, err) } @@ -212,7 +220,7 @@ func (m *Manager) processPendingJobs(ctx context.Context, pendingJobs []model.So } for _, job := range responses.Jobs { - var response notifierResponse + var response NotifierResponse var jobStatus model.SourceJobStatus if err = json.Unmarshal(job.Payload, &response); err != nil { @@ -221,9 +229,9 @@ func (m *Manager) processPendingJobs(ctx context.Context, pendingJobs []model.So if jobStatus, err = model.FromSourceJobStatus(string(job.Status)); err != nil { return fmt.Errorf("invalid job status %s for source job %d: %w", job.Status, job.ID, err) } - if pj, ok := pendingJobsMap[response.Id]; ok { - pj.Status = jobStatus - pj.Error = job.Error + if pendingJob, ok := pendingJobsMap[response.ID]; ok { + pendingJob.Status = jobStatus + pendingJob.Error = job.Error } } diff --git a/warehouse/source/source_test.go b/warehouse/source/source_test.go index bde6d9c32b..9aa96a61b1 100644 --- a/warehouse/source/source_test.go +++ b/warehouse/source/source_test.go @@ -226,7 +226,6 @@ func TestSource(t *testing.T) { Status: notifier.Succeeded, }, }, - Err: nil, } close(response) publishResponse := make(chan chan *notifier.PublishResponse, 1) diff --git a/warehouse/source/types.go b/warehouse/source/types.go index 6f8b211b2c..9a6e88916e 100644 --- a/warehouse/source/types.go +++ b/warehouse/source/types.go @@ -2,6 +2,7 @@ package source import ( "context" + "encoding/json" "errors" "github.com/rudderlabs/rudder-server/services/notifier" @@ -33,8 +34,18 @@ type jobStatusResponse struct { Err string } -type notifierResponse struct { - Id int64 `json:"id"` +type NotifierRequest struct { + ID int64 `json:"id"` + SourceID string `json:"source_id"` + DestinationID string `json:"destination_id"` + WorkspaceID string `json:"workspace_id"` + TableName string `json:"tablename"` + JobType string `json:"async_job_type"` + MetaData json.RawMessage `json:"metadata"` +} + +type NotifierResponse struct { + ID int64 `json:"id"` } type publisher interface { From 59b73c18ac993e26808ec5a78b72251e4696158e Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Tue, 31 Oct 2023 17:08:12 +0530 Subject: [PATCH 6/7] chore: some minor chagnes --- warehouse/internal/repo/source.go | 6 +++--- warehouse/source/source.go | 20 +++++++++++++------- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/warehouse/internal/repo/source.go b/warehouse/internal/repo/source.go index ebcc882880..fc2cac0b3c 100644 --- a/warehouse/internal/repo/source.go +++ b/warehouse/internal/repo/source.go @@ -237,7 +237,7 @@ func (s *Source) OnUpdateSuccess(ctx context.Context, id int64) error { WHERE id = $3; `, - model.SourceJobStatusSucceeded, + model.SourceJobStatusSucceeded.String(), s.now(), id, ) @@ -270,8 +270,8 @@ func (s *Source) OnUpdateFailure(ctx context.Context, id int64, error error, max id = $6; `, maxAttempt, - model.SourceJobStatusAborted, - model.SourceJobStatusFailed, + model.SourceJobStatusAborted.String(), + model.SourceJobStatusFailed.String(), s.now(), error.Error(), id, diff --git a/warehouse/source/source.go b/warehouse/source/source.go index 3723105c0c..4d0a11488b 100644 --- a/warehouse/source/source.go +++ b/warehouse/source/source.go @@ -8,6 +8,8 @@ import ( "strings" "time" + "github.com/lib/pq" + "github.com/samber/lo" "github.com/rudderlabs/rudder-server/services/notifier" @@ -60,11 +62,8 @@ func New(conf *config.Config, log logger.Logger, db *sqlmw.DB, publisher publish } func (m *Manager) InsertJobs(ctx context.Context, payload insertJobRequest) ([]int64, error) { - var jobType model.SourceJobType - switch payload.JobType { - case model.SourceJobTypeDeleteByJobRunID.String(): - jobType = model.SourceJobTypeDeleteByJobRunID - default: + jobType, err := model.FromSourceJobType(payload.JobType) + if err != nil { return nil, fmt.Errorf("invalid job type %s", payload.JobType) } @@ -127,8 +126,15 @@ func (m *Manager) Run(ctx context.Context) error { return fmt.Errorf("resetting source jobs with error %w", err) } - if err := m.process(ctx); err != nil && !errors.Is(err, context.Canceled) { - return fmt.Errorf("processing source jobs with error %w", err) + if err := m.process(ctx); err != nil { + var pqErr *pq.Error + + switch { + case errors.Is(err, context.Canceled), errors.Is(err, context.DeadlineExceeded), errors.As(err, &pqErr) && pqErr.Code == "57014": + return nil + default: + return fmt.Errorf("processing source jobs with error %w", err) + } } return nil } From 691c6559d67102428b1e0411345d0c1163c9d310 Mon Sep 17 00:00:00 2001 From: achettyiitr Date: Wed, 1 Nov 2023 00:04:38 +0530 Subject: [PATCH 7/7] chore: added some more test cases --- warehouse/internal/model/source_test.go | 114 ++++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 warehouse/internal/model/source_test.go diff --git a/warehouse/internal/model/source_test.go b/warehouse/internal/model/source_test.go new file mode 100644 index 0000000000..49896f9999 --- /dev/null +++ b/warehouse/internal/model/source_test.go @@ -0,0 +1,114 @@ +package model + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFromSourceJobType(t *testing.T) { + testCases := []struct { + name string + jobType string + expected SourceJobType + wantErr error + }{ + { + name: "delete bv job run id", + jobType: "deletebyjobrunid", + expected: SourceJobTypeDeleteByJobRunID, + wantErr: nil, + }, + { + name: "invalid", + jobType: "invalid", + expected: nil, + wantErr: fmt.Errorf("invalid job type %s", "invalid"), + }, + { + name: "empty", + jobType: "", + expected: nil, + wantErr: fmt.Errorf("invalid job type %s", ""), + }, + { + name: "nil", + jobType: "", + expected: nil, + wantErr: fmt.Errorf("invalid job type %s", ""), + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + jobType, err := FromSourceJobType(tc.jobType) + if tc.wantErr != nil { + require.Equal(t, tc.wantErr, err) + require.Nil(t, jobType) + return + } + require.NoError(t, err) + require.Equal(t, tc.expected, jobType) + }) + } +} + +func TestFromSourceJobStatus(t *testing.T) { + testCases := []struct { + name string + status string + expected SourceJobStatus + wantError error + }{ + { + name: "waiting", + status: "waiting", + expected: SourceJobStatusWaiting, + wantError: nil, + }, + { + name: "executing", + status: "executing", + expected: SourceJobStatusExecuting, + wantError: nil, + }, + { + name: "failed", + status: "failed", + expected: SourceJobStatusFailed, + wantError: nil, + }, + { + name: "aborted", + status: "aborted", + expected: SourceJobStatusAborted, + wantError: nil, + }, + { + name: "succeeded", + status: "succeeded", + expected: SourceJobStatusSucceeded, + wantError: nil, + }, + { + name: "invalid", + status: "invalid", + expected: nil, + wantError: fmt.Errorf("invalid job status %s", "invalid"), + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + jobStatus, err := FromSourceJobStatus(tc.status) + if tc.wantError != nil { + require.Equal(t, tc.wantError, err) + require.Nil(t, jobStatus) + return + } + require.NoError(t, err) + require.Equal(t, tc.expected, jobStatus) + }) + } +}