github.com/pachyderm/pachyderm@v1.13.4/src/server/worker/pipeline/service/service.go (about)

     1  package service
     2  
     3  import (
     4  	"context"
     5  
     6  	"golang.org/x/sync/errgroup"
     7  
     8  	"github.com/pachyderm/pachyderm/src/client"
     9  	"github.com/pachyderm/pachyderm/src/client/pfs"
    10  	"github.com/pachyderm/pachyderm/src/client/pkg/errors"
    11  	"github.com/pachyderm/pachyderm/src/client/pps"
    12  	"github.com/pachyderm/pachyderm/src/server/pkg/ppsconsts"
    13  	"github.com/pachyderm/pachyderm/src/server/pkg/ppsutil"
    14  	"github.com/pachyderm/pachyderm/src/server/worker/common"
    15  	"github.com/pachyderm/pachyderm/src/server/worker/datum"
    16  	"github.com/pachyderm/pachyderm/src/server/worker/driver"
    17  	"github.com/pachyderm/pachyderm/src/server/worker/logs"
    18  	"github.com/pachyderm/pachyderm/src/server/worker/pipeline"
    19  )
    20  
    21  // Repeatedly runs the given callback with the latest commit for the pipeline.
    22  // The given context will be canceled if a newer commit is ready, then this will
    23  // wait for the previous callback to return before calling the callback again
    24  // with the latest commit.
    25  func forLatestCommit(
    26  	pachClient *client.APIClient,
    27  	pipelineInfo *pps.PipelineInfo,
    28  	logger logs.TaggedLogger,
    29  	cb func(context.Context, *pfs.CommitInfo) error,
    30  ) error {
    31  	// These are used to cancel the existing service and wait for it to finish
    32  	var cancel func()
    33  	var eg *errgroup.Group
    34  
    35  	return pachClient.SubscribeCommitF(
    36  		pipelineInfo.Pipeline.Name,
    37  		"",
    38  		client.NewCommitProvenance(ppsconsts.SpecRepo, pipelineInfo.Pipeline.Name, pipelineInfo.SpecCommit.ID),
    39  		"",
    40  		pfs.CommitState_READY,
    41  		func(ci *pfs.CommitInfo) error {
    42  			if cancel != nil {
    43  				logger.Logf("canceling previous service, new commit ready")
    44  				cancel()
    45  				if err := eg.Wait(); err != nil && !errors.Is(err, context.Canceled) {
    46  					return err
    47  				} else if common.IsDone(pachClient.Ctx()) {
    48  					return pachClient.Ctx().Err()
    49  				}
    50  			}
    51  
    52  			logger.Logf("starting new service, commit: %s", ci.Commit.ID)
    53  
    54  			var ctx context.Context
    55  			ctx, cancel = context.WithCancel(pachClient.Ctx())
    56  			eg, ctx = errgroup.WithContext(ctx)
    57  			eg.Go(func() error { return cb(ctx, ci) })
    58  
    59  			return nil
    60  		},
    61  	)
    62  }
    63  
    64  // Run will run a service pipeline until the driver is canceled.
    65  func Run(driver driver.Driver, logger logs.TaggedLogger) error {
    66  	pachClient := driver.PachClient()
    67  	pipelineInfo := driver.PipelineInfo()
    68  
    69  	// The serviceCtx is only used for canceling user code (due to a new output
    70  	// commit being ready)
    71  	return forLatestCommit(pachClient, driver.PipelineInfo(), logger, func(serviceCtx context.Context, commitInfo *pfs.CommitInfo) error {
    72  		// Create a job document matching the service's output commit
    73  		jobInput := ppsutil.JobInput(pipelineInfo, commitInfo)
    74  		job, err := pachClient.CreateJob(pipelineInfo.Pipeline.Name, commitInfo.Commit, nil)
    75  		if err != nil {
    76  			return err
    77  		}
    78  		logger := logger.WithJob(job.ID)
    79  
    80  		dit, err := datum.NewIterator(pachClient, jobInput)
    81  		if err != nil {
    82  			return err
    83  		}
    84  		if dit.Len() != 1 {
    85  			return errors.New("services must have a single datum")
    86  		}
    87  		inputs := dit.DatumN(0)
    88  		logger = logger.WithData(inputs)
    89  
    90  		// TODO: do something with stats? - this isn't an output repo so there's nowhere to put them
    91  		_, err = driver.WithData(inputs, nil, logger, func(dir string, stats *pps.ProcessStats) error {
    92  			if err := driver.UpdateJobState(job.ID, pps.JobState_JOB_RUNNING, ""); err != nil {
    93  				logger.Logf("error updating job state: %+v", err)
    94  			}
    95  
    96  			eg, serviceCtx := errgroup.WithContext(serviceCtx)
    97  			eg.Go(func() error {
    98  				return driver.WithActiveData(inputs, dir, func() error {
    99  					return pipeline.RunUserCode(driver.WithContext(serviceCtx), logger, nil, inputs)
   100  				})
   101  			})
   102  			if pipelineInfo.Spout != nil {
   103  				eg.Go(func() error { return pipeline.ReceiveSpout(serviceCtx, pachClient, pipelineInfo, logger) })
   104  			}
   105  
   106  			if err := eg.Wait(); err != nil {
   107  				logger.Logf("error running user code: %+v", err)
   108  			}
   109  
   110  			// Only want to update this stuff if we were canceled due to a new commit
   111  			if common.IsDone(serviceCtx) {
   112  				// TODO: do this in a transaction
   113  				if err := driver.UpdateJobState(job.ID, pps.JobState_JOB_SUCCESS, ""); err != nil {
   114  					logger.Logf("error updating job progress: %+v", err)
   115  				}
   116  				if err := pachClient.FinishCommit(commitInfo.Commit.Repo.Name, commitInfo.Commit.ID); err != nil {
   117  					logger.Logf("could not finish output commit: %v", err)
   118  				}
   119  			}
   120  			return nil
   121  		})
   122  		return err
   123  	})
   124  }