github.com/pachyderm/pachyderm@v1.13.4/src/server/worker/driver/driver.go (about) 1 package driver 2 3 import ( 4 "bufio" 5 "bytes" 6 "context" 7 "encoding/json" 8 "fmt" 9 "io" 10 "io/ioutil" 11 "log" 12 "os" 13 "os/user" 14 "path/filepath" 15 "strconv" 16 "strings" 17 "sync" 18 "sync/atomic" 19 "syscall" 20 "time" 21 "unicode/utf8" 22 23 etcd "github.com/coreos/etcd/clientv3" 24 "github.com/gogo/protobuf/types" 25 "github.com/prometheus/client_golang/prometheus" 26 "gopkg.in/go-playground/webhooks.v5/github" 27 "gopkg.in/src-d/go-git.v4" 28 gitPlumbing "gopkg.in/src-d/go-git.v4/plumbing" 29 30 "github.com/pachyderm/pachyderm/src/client" 31 "github.com/pachyderm/pachyderm/src/client/enterprise" 32 "github.com/pachyderm/pachyderm/src/client/pfs" 33 "github.com/pachyderm/pachyderm/src/client/pkg/errors" 34 "github.com/pachyderm/pachyderm/src/client/pkg/grpcutil" 35 "github.com/pachyderm/pachyderm/src/client/pps" 36 col "github.com/pachyderm/pachyderm/src/server/pkg/collection" 37 "github.com/pachyderm/pachyderm/src/server/pkg/errutil" 38 "github.com/pachyderm/pachyderm/src/server/pkg/exec" 39 "github.com/pachyderm/pachyderm/src/server/pkg/hashtree" 40 "github.com/pachyderm/pachyderm/src/server/pkg/obj" 41 "github.com/pachyderm/pachyderm/src/server/pkg/ppsconsts" 42 "github.com/pachyderm/pachyderm/src/server/pkg/ppsdb" 43 "github.com/pachyderm/pachyderm/src/server/pkg/ppsutil" 44 filesync "github.com/pachyderm/pachyderm/src/server/pkg/sync" 45 "github.com/pachyderm/pachyderm/src/server/pkg/uuid" 46 "github.com/pachyderm/pachyderm/src/server/pkg/work" 47 "github.com/pachyderm/pachyderm/src/server/worker/cache" 48 "github.com/pachyderm/pachyderm/src/server/worker/common" 49 "github.com/pachyderm/pachyderm/src/server/worker/logs" 50 "github.com/pachyderm/pachyderm/src/server/worker/stats" 51 ) 52 53 const ( 54 // The maximum number of concurrent download/upload operations 55 concurrency = 100 56 ) 57 58 var ( 59 errSpecialFile = errors.New("cannot upload special file") 60 ) 61 62 // WorkNamespace returns the namespace used by the work package for this 63 // pipeline. 64 func WorkNamespace(pipelineInfo *pps.PipelineInfo) string { 65 return fmt.Sprintf("/pipeline-%s/v%d", pipelineInfo.Pipeline.Name, pipelineInfo.Version) 66 } 67 68 // Driver provides an interface for common functions needed by worker code, and 69 // captures the relevant objects necessary to provide these functions so that 70 // users do not need to keep track of as many variables. In addition, this 71 // interface can be used to mock out external calls to make unit-testing 72 // simpler. 73 type Driver interface { 74 Jobs() col.Collection 75 Pipelines() col.Collection 76 77 NewTaskWorker() *work.Worker 78 NewTaskQueue() (*work.TaskQueue, error) 79 80 // Returns the PipelineInfo for the pipeline that this worker belongs to 81 PipelineInfo() *pps.PipelineInfo 82 83 // Returns the kubernetes namespace that the worker is deployed in 84 Namespace() string 85 86 // Returns the path that will contain the input filesets for the job 87 InputDir() string 88 89 // Returns the pachd API client for the driver 90 PachClient() *client.APIClient 91 92 // Returns the number of workers to be used 93 ExpectedNumWorkers() (int64, error) 94 95 // Returns the number of hashtree shards for the pipeline 96 NumShards() int64 97 98 // WithContext clones the current driver and applies the context to its 99 // pachClient. The pachClient context will be used for other blocking 100 // operations as well. 101 WithContext(context.Context) Driver 102 103 // WithData prepares the current node the code is running on to run a piece 104 // of user code by downloading the specified data, and cleans up afterwards. 105 // The temporary scratch directory that the data is stored in will be passed 106 // to the given callback. 107 WithData([]*common.Input, *hashtree.Ordered, logs.TaggedLogger, func(string, *pps.ProcessStats) error) (*pps.ProcessStats, error) 108 109 // WithActiveData swaps the given scratch directory into the 'active' input 110 // directory used when running user code. This also locks a mutex so that no 111 // two datums can be active concurrently. 112 WithActiveData([]*common.Input, string, func() error) error 113 114 // UserCodeEnv returns the set of environment variables to construct when 115 // launching the configured user process. 116 UserCodeEnv(string, *pfs.Commit, []*common.Input) []string 117 118 // RunUserCode links a specific scratch space for the active input/output 119 // data, then runs the pipeline's configured code. It uses a mutex to enforce 120 // that this is not done concurrently, and may block. 121 RunUserCode(logs.TaggedLogger, []string, *pps.ProcessStats, *types.Duration) error 122 123 // RunUserErrorHandlingCode runs the pipeline's configured error handling code 124 RunUserErrorHandlingCode(logs.TaggedLogger, []string, *pps.ProcessStats, *types.Duration) error 125 126 // TODO: provide a more generic interface for modifying jobs, and 127 // some quality-of-life functions for common operations. 128 DeleteJob(col.STM, *pps.EtcdJobInfo) error 129 UpdateJobState(string, pps.JobState, string) error 130 131 // UploadOutput uploads the stats hashtree and pfs output directory to object 132 // storage and returns a buffer of the serialized hashtree 133 UploadOutput(string, string, logs.TaggedLogger, []*common.Input, *pps.ProcessStats, *hashtree.Ordered) ([]byte, error) 134 135 // TODO: figure out how to not expose this 136 ReportUploadStats(time.Time, *pps.ProcessStats, logs.TaggedLogger) 137 138 // TODO: figure out how to not expose this - currently only used for a few 139 // operations in the map spawner 140 NewSTM(func(col.STM) error) (*etcd.TxnResponse, error) 141 142 // These caches are used for storing and merging hashtrees from jobs until the 143 // job is complete 144 ChunkCaches() cache.WorkerCache 145 ChunkStatsCaches() cache.WorkerCache 146 147 // WithDatumCache calls the given callback with two hashtree merge caches, one 148 // for datums and one for datum stats. The lifetime of these caches will be 149 // bound to the callback, and any resources will be cleaned up upon return. 150 WithDatumCache(func(*hashtree.MergeCache, *hashtree.MergeCache) error) error 151 152 Egress(commit *pfs.Commit, egressURL string) error 153 } 154 155 type driver struct { 156 pipelineInfo *pps.PipelineInfo 157 pachClient *client.APIClient 158 etcdClient *etcd.Client 159 etcdPrefix string 160 activeDataMutex *sync.Mutex 161 162 jobs col.Collection 163 164 pipelines col.Collection 165 166 numShards int64 167 168 namespace string 169 170 // User and group IDs used for running user code, determined in the constructor 171 uid *uint32 172 gid *uint32 173 174 // We only export application statistics if enterprise is enabled 175 exportStats bool 176 177 // The root directory to use when setting the user code path. This is normally 178 // "/", but can be overridden by tests. 179 rootDir string 180 181 // The directory to store input data - this is typically static but can be 182 // overridden by tests. 183 inputDir string 184 185 // The directory to store hashtrees in. This will be cleared when starting to 186 // avoid artifacts from previous runs. 187 hashtreeDir string 188 189 // These caches are used for storing and merging hashtrees from jobs until the 190 // job is complete 191 chunkCaches, chunkStatsCaches cache.WorkerCache 192 } 193 194 // NewDriver constructs a Driver object using the given clients and pipeline 195 // settings. It makes blocking calls to determine the user/group to use with 196 // the user code on the current worker node, as well as determining if 197 // enterprise features are activated (for exporting stats). 198 func NewDriver( 199 pipelineInfo *pps.PipelineInfo, 200 pachClient *client.APIClient, 201 etcdClient *etcd.Client, 202 etcdPrefix string, 203 hashtreePath string, 204 rootPath string, 205 namespace string, 206 ) (Driver, error) { 207 208 pfsPath := filepath.Join(rootPath, client.PPSInputPrefix) 209 chunkCachePath := filepath.Join(hashtreePath, "chunk") 210 chunkStatsCachePath := filepath.Join(hashtreePath, "chunkStats") 211 212 // Delete the hashtree path (if it exists) in case it is left over from a previous run 213 if err := os.RemoveAll(chunkCachePath); err != nil { 214 return nil, errors.EnsureStack(err) 215 } 216 if err := os.RemoveAll(chunkStatsCachePath); err != nil { 217 return nil, errors.EnsureStack(err) 218 } 219 220 if err := os.MkdirAll(pfsPath, 0777); err != nil { 221 return nil, errors.EnsureStack(err) 222 } 223 if err := os.MkdirAll(chunkCachePath, 0777); err != nil { 224 return nil, errors.EnsureStack(err) 225 } 226 if err := os.MkdirAll(chunkStatsCachePath, 0777); err != nil { 227 return nil, errors.EnsureStack(err) 228 } 229 230 numShards, err := ppsutil.GetExpectedNumHashtrees(pipelineInfo.HashtreeSpec) 231 if err != nil { 232 logs.NewStatlessLogger(pipelineInfo).Logf("error getting number of shards, default to 1 shard: %v", err) 233 numShards = 1 234 } 235 236 result := &driver{ 237 pipelineInfo: pipelineInfo, 238 pachClient: pachClient, 239 etcdClient: etcdClient, 240 etcdPrefix: etcdPrefix, 241 activeDataMutex: &sync.Mutex{}, 242 jobs: ppsdb.Jobs(etcdClient, etcdPrefix), 243 pipelines: ppsdb.Pipelines(etcdClient, etcdPrefix), 244 numShards: numShards, 245 rootDir: rootPath, 246 inputDir: pfsPath, 247 hashtreeDir: hashtreePath, 248 chunkCaches: cache.NewWorkerCache(chunkCachePath), 249 chunkStatsCaches: cache.NewWorkerCache(chunkStatsCachePath), 250 namespace: namespace, 251 } 252 253 if pipelineInfo.Transform.User != "" { 254 user, err := lookupDockerUser(pipelineInfo.Transform.User) 255 if err != nil && !errors.Is(err, os.ErrNotExist) { 256 return nil, errors.EnsureStack(err) 257 } 258 // If `user` is `nil`, `uid` and `gid` will get set, and we won't 259 // customize the user that executes the worker process. 260 if user != nil { // user is nil when err is an os.ErrNotExist is true in which case we use root 261 uid, err := strconv.ParseUint(user.Uid, 10, 32) 262 if err != nil { 263 return nil, errors.EnsureStack(err) 264 } 265 uid32 := uint32(uid) 266 result.uid = &uid32 267 gid, err := strconv.ParseUint(user.Gid, 10, 32) 268 if err != nil { 269 return nil, errors.EnsureStack(err) 270 } 271 gid32 := uint32(gid) 272 result.gid = &gid32 273 } 274 } 275 276 if resp, err := pachClient.Enterprise.GetState(context.Background(), &enterprise.GetStateRequest{}); err != nil { 277 logs.NewStatlessLogger(pipelineInfo).Logf("failed to get enterprise state with error: %v\n", err) 278 } else { 279 result.exportStats = resp.State == enterprise.State_ACTIVE 280 } 281 282 return result, nil 283 } 284 285 // lookupDockerUser looks up users given the argument to a Dockerfile USER directive. 286 // According to Docker's docs this directive looks like: 287 // USER <user>[:<group>] or 288 // USER <UID>[:<GID>] 289 func lookupDockerUser(userArg string) (_ *user.User, retErr error) { 290 userParts := strings.Split(userArg, ":") 291 userOrUID := userParts[0] 292 groupOrGID := "" 293 if len(userParts) > 1 { 294 groupOrGID = userParts[1] 295 } 296 passwd, err := os.Open("/etc/passwd") 297 if err != nil { 298 return nil, errors.EnsureStack(err) 299 } 300 defer func() { 301 if err := passwd.Close(); err != nil && retErr == nil { 302 retErr = errors.EnsureStack(err) 303 } 304 }() 305 scanner := bufio.NewScanner(passwd) 306 for scanner.Scan() { 307 parts := strings.Split(scanner.Text(), ":") 308 if parts[0] == userOrUID || parts[2] == userOrUID { 309 result := &user.User{ 310 Username: parts[0], 311 Uid: parts[2], 312 Gid: parts[3], 313 Name: parts[4], 314 HomeDir: parts[5], 315 } 316 if groupOrGID != "" { 317 if parts[0] == userOrUID { 318 // groupOrGid is a group 319 group, err := lookupGroup(groupOrGID) 320 if err != nil { 321 return nil, errors.EnsureStack(err) 322 } 323 result.Gid = group.Gid 324 } else { 325 // groupOrGid is a gid 326 result.Gid = groupOrGID 327 } 328 } 329 return result, nil 330 } 331 } 332 if err := scanner.Err(); err != nil { 333 log.Fatal(err) 334 } 335 return nil, errors.Errorf("user %s not found", userArg) 336 } 337 338 func lookupGroup(group string) (_ *user.Group, retErr error) { 339 groupFile, err := os.Open("/etc/group") 340 if err != nil { 341 return nil, errors.EnsureStack(err) 342 } 343 defer func() { 344 if err := groupFile.Close(); err != nil && retErr == nil { 345 retErr = errors.EnsureStack(err) 346 } 347 }() 348 scanner := bufio.NewScanner(groupFile) 349 for scanner.Scan() { 350 parts := strings.Split(scanner.Text(), ":") 351 if parts[0] == group { 352 return &user.Group{ 353 Gid: parts[2], 354 Name: parts[0], 355 }, nil 356 } 357 } 358 return nil, errors.Errorf("group %s not found", group) 359 } 360 361 func (d *driver) WithContext(ctx context.Context) Driver { 362 result := &driver{} 363 *result = *d 364 result.pachClient = result.pachClient.WithCtx(ctx) 365 return result 366 } 367 368 func (d *driver) Jobs() col.Collection { 369 return d.jobs 370 } 371 372 func (d *driver) Pipelines() col.Collection { 373 return d.pipelines 374 } 375 376 func (d *driver) NewTaskWorker() *work.Worker { 377 return work.NewWorker(d.etcdClient, d.etcdPrefix, WorkNamespace(d.pipelineInfo)) 378 } 379 380 func (d *driver) NewTaskQueue() (*work.TaskQueue, error) { 381 return work.NewTaskQueue(d.PachClient().Ctx(), d.etcdClient, d.etcdPrefix, WorkNamespace(d.pipelineInfo)) 382 } 383 384 func (d *driver) ExpectedNumWorkers() (int64, error) { 385 pipelinePtr := &pps.EtcdPipelineInfo{} 386 if err := d.Pipelines().ReadOnly(d.PachClient().Ctx()).Get(d.PipelineInfo().Pipeline.Name, pipelinePtr); err != nil { 387 return 0, errors.EnsureStack(err) 388 } 389 numWorkers := pipelinePtr.Parallelism 390 if numWorkers == 0 { 391 numWorkers = 1 392 } 393 return int64(numWorkers), nil 394 } 395 396 func (d *driver) NumShards() int64 { 397 return d.numShards 398 } 399 400 func (d *driver) PipelineInfo() *pps.PipelineInfo { 401 return d.pipelineInfo 402 } 403 404 func (d *driver) Namespace() string { 405 return d.namespace 406 } 407 408 func (d *driver) InputDir() string { 409 return d.inputDir 410 } 411 412 func (d *driver) PachClient() *client.APIClient { 413 return d.pachClient 414 } 415 416 func (d *driver) ChunkCaches() cache.WorkerCache { 417 return d.chunkCaches 418 } 419 420 func (d *driver) ChunkStatsCaches() cache.WorkerCache { 421 return d.chunkStatsCaches 422 } 423 424 // This is broken out into its own function because its scope is small and it 425 // can easily be used by the mock driver for testing purposes. 426 func withDatumCache(storageRoot string, cb func(*hashtree.MergeCache, *hashtree.MergeCache) error) (retErr error) { 427 cacheID := uuid.NewWithoutDashes() 428 429 datumCache, err := hashtree.NewMergeCache(filepath.Join(storageRoot, "datum", cacheID)) 430 if err != nil { 431 return err 432 } 433 defer func() { 434 if err := datumCache.Close(); retErr == nil { 435 retErr = err 436 } 437 }() 438 439 datumStatsCache, err := hashtree.NewMergeCache(filepath.Join(storageRoot, "datumStats", cacheID)) 440 if err != nil { 441 return err 442 } 443 defer func() { 444 if err := datumStatsCache.Close(); retErr == nil { 445 retErr = err 446 } 447 }() 448 449 return cb(datumCache, datumStatsCache) 450 } 451 452 func (d *driver) WithDatumCache(cb func(*hashtree.MergeCache, *hashtree.MergeCache) error) (retErr error) { 453 return withDatumCache(d.hashtreeDir, cb) 454 } 455 456 func (d *driver) NewSTM(cb func(col.STM) error) (*etcd.TxnResponse, error) { 457 return col.NewSTM(d.pachClient.Ctx(), d.etcdClient, cb) 458 } 459 460 func (d *driver) WithData( 461 inputs []*common.Input, 462 inputTree *hashtree.Ordered, 463 logger logs.TaggedLogger, 464 cb func(string, *pps.ProcessStats) error, 465 ) (retStats *pps.ProcessStats, retErr error) { 466 puller := filesync.NewPuller() 467 stats := &pps.ProcessStats{} 468 469 // Download input data into a temporary directory 470 // This can be interrupted via the pachClient using driver.WithContext 471 dir, err := d.downloadData(logger, inputs, puller, stats, inputTree) 472 // We run these cleanup functions no matter what, so that if 473 // downloadData partially succeeded, we still clean up the resources. 474 defer func() { 475 if err := os.RemoveAll(dir); err != nil && retErr == nil { 476 retErr = errors.EnsureStack(err) 477 } 478 }() 479 // It's important that we run puller.CleanUp before os.RemoveAll, 480 // because otherwise puller.Cleanup might try to open pipes that have 481 // been deleted. 482 defer func() { 483 if _, err := puller.CleanUp(); err != nil && retErr == nil { 484 retErr = errors.EnsureStack(err) 485 } 486 }() 487 if err != nil { 488 return stats, errors.EnsureStack(err) 489 } 490 if err := os.MkdirAll(d.InputDir(), 0777); err != nil { 491 return stats, errors.EnsureStack(err) 492 } 493 // If the pipeline spec set a custom user to execute the process, make sure 494 // the input directory and its content are owned by it 495 if d.uid != nil && d.gid != nil { 496 filepath.Walk(dir, func(name string, info os.FileInfo, err error) error { 497 if err == nil { 498 err = os.Chown(name, int(*d.uid), int(*d.gid)) 499 } 500 return errors.EnsureStack(err) 501 }) 502 } 503 504 if err := cb(dir, stats); err != nil { 505 return stats, err 506 } 507 508 // CleanUp is idempotent so we can call it however many times we want. 509 // The reason we are calling it here is that the puller could've 510 // encountered an error as it was lazily loading files, in which case 511 // the output might be invalid since as far as the user's code is 512 // concerned, they might've just seen an empty or partially completed 513 // file. 514 // TODO: do we really need two puller.CleanUps? 515 downSize, err := puller.CleanUp() 516 if err != nil { 517 logger.Logf("puller encountered an error while cleaning up: %v", err) 518 return stats, errors.EnsureStack(err) 519 } 520 521 atomic.AddUint64(&stats.DownloadBytes, uint64(downSize)) 522 d.reportDownloadSizeStats(float64(downSize), logger) 523 return stats, nil 524 } 525 526 func (d *driver) downloadData( 527 logger logs.TaggedLogger, 528 inputs []*common.Input, 529 puller *filesync.Puller, 530 stats *pps.ProcessStats, 531 statsTree *hashtree.Ordered, 532 ) (_ string, retErr error) { 533 defer d.reportDownloadTimeStats(time.Now(), stats, logger) 534 logger.Logf("starting to download data") 535 defer func(start time.Time) { 536 if retErr != nil { 537 logger.Logf("errored downloading data after %v: %v", time.Since(start), retErr) 538 } else { 539 logger.Logf("finished downloading data after %v", time.Since(start)) 540 } 541 }(time.Now()) 542 543 // The scratch space is where Pachyderm stores downloaded and output data, which is 544 // then symlinked into place for the pipeline. 545 scratchPath := filepath.Join(d.InputDir(), client.PPSScratchSpace, uuid.NewWithoutDashes()) 546 547 // Clean up any files if an error occurs 548 defer func() { 549 if retErr != nil { 550 os.RemoveAll(scratchPath) 551 } 552 }() 553 554 outPath := filepath.Join(scratchPath, "out") 555 // TODO: move this up into spout code 556 if d.pipelineInfo.Spout != nil { 557 // Spouts need to create a named pipe at /pfs/out 558 if err := os.MkdirAll(filepath.Dir(outPath), 0700); err != nil { 559 return "", errors.EnsureStack(err) 560 } 561 // Fifos don't exist on windows (where we may run this code in tests), so 562 // this function is defined in a conditional-build file 563 if err := createSpoutFifo(outPath); err != nil { 564 return "", err 565 } 566 if d.PipelineInfo().Spout.Marker != "" { 567 // check if we have a marker file in the /pfs/out directory 568 _, err := d.PachClient().InspectFile(d.PipelineInfo().Pipeline.Name, ppsconsts.SpoutMarkerBranch, d.PipelineInfo().Spout.Marker) 569 if err != nil { 570 // if this fails because there's no head commit on the marker branch, then we don't want to pull the marker, but it's also not an error 571 if !strings.Contains(err.Error(), "no head") { 572 // if it fails for some other reason, then fail 573 return "", errors.EnsureStack(err) 574 } 575 } else { 576 // the file might be in the spout marker directory, and so we'll try pulling it from there 577 if err := puller.Pull( 578 d.PachClient(), 579 filepath.Join(scratchPath, d.PipelineInfo().Spout.Marker), 580 d.PipelineInfo().Pipeline.Name, 581 ppsconsts.SpoutMarkerBranch, 582 "/"+d.PipelineInfo().Spout.Marker, 583 false, 584 false, 585 concurrency, 586 nil, 587 "", 588 ); err != nil { 589 // this might fail if the marker branch hasn't been created, so check for that 590 if err == nil || !strings.Contains(err.Error(), "branches") || !errutil.IsNotFoundError(err) { 591 return "", errors.EnsureStack(err) 592 } 593 // if it just hasn't been created yet, that's fine and we should just continue as normal 594 } 595 } 596 } 597 } else if !d.PipelineInfo().S3Out { 598 // Create output directory (typically /pfs/out) 599 if err := os.MkdirAll(outPath, 0777); err != nil { 600 return "", errors.Wrapf(err, "couldn't create %q", outPath) 601 } 602 } 603 for _, input := range inputs { 604 if input.GitURL != "" { 605 if err := d.downloadGitData(scratchPath, input); err != nil { 606 return "", err 607 } 608 continue 609 } 610 if input.S3 { 611 continue // don't download any data 612 } 613 file := input.FileInfo.File 614 fullInputPath := filepath.Join(scratchPath, input.Name, file.Path) 615 var statsRoot string 616 if statsTree != nil { 617 statsRoot = filepath.Join(input.Name, file.Path) 618 parent, _ := filepath.Split(statsRoot) 619 statsTree.MkdirAll(parent) 620 } 621 if err := puller.Pull( 622 d.pachClient, 623 fullInputPath, 624 file.Commit.Repo.Name, 625 file.Commit.ID, 626 file.Path, 627 input.Lazy, 628 input.EmptyFiles, 629 concurrency, 630 statsTree, 631 statsRoot, 632 ); err != nil { 633 return "", errors.EnsureStack(err) 634 } 635 } 636 return scratchPath, nil 637 } 638 639 func (d *driver) downloadGitData(scratchPath string, input *common.Input) error { 640 file := input.FileInfo.File 641 642 var rawJSON bytes.Buffer 643 err := d.pachClient.GetFile(file.Commit.Repo.Name, file.Commit.ID, file.Path, 0, 0, &rawJSON) 644 if err != nil { 645 return errors.EnsureStack(err) 646 } 647 648 var payload github.PushPayload 649 err = json.Unmarshal(rawJSON.Bytes(), &payload) 650 if err != nil { 651 return errors.EnsureStack(err) 652 } 653 654 if payload.Repository.CloneURL == "" { 655 return errors.New("git hook payload does not specify the upstream URL") 656 } else if payload.Ref == "" { 657 return errors.New("git hook payload does not specify the updated ref") 658 } else if payload.After == "" { 659 return errors.New("git hook payload does not specify the commit SHA") 660 } 661 662 // Clone checks out a reference, not a SHA. Github does not support fetching 663 // an individual SHA. 664 remoteURL := payload.Repository.CloneURL 665 gitRepo, err := git.PlainCloneContext( 666 d.pachClient.Ctx(), 667 filepath.Join(scratchPath, input.Name), 668 false, 669 &git.CloneOptions{ 670 URL: remoteURL, 671 SingleBranch: true, 672 ReferenceName: gitPlumbing.ReferenceName(payload.Ref), 673 }, 674 ) 675 if err != nil { 676 return errors.Wrapf(err, "error fetching repo %v with ref %v from URL %v", input.Name, payload.Ref, remoteURL) 677 } 678 679 wt, err := gitRepo.Worktree() 680 if err != nil { 681 return errors.EnsureStack(err) 682 } 683 684 sha := payload.After 685 err = wt.Checkout(&git.CheckoutOptions{Hash: gitPlumbing.NewHash(sha)}) 686 if err != nil { 687 return errors.Wrapf(err, "error checking out SHA %v for repo %v", sha, input.Name) 688 } 689 690 // go-git will silently fail to checkout an invalid SHA and leave the HEAD at 691 // the selected ref. Verify that we are now on the correct SHA 692 rev, err := gitRepo.ResolveRevision("HEAD") 693 if err != nil { 694 return errors.Wrapf(err, "failed to inspect HEAD SHA for repo %v", input.Name) 695 } else if rev.String() != sha { 696 return errors.Errorf("could not find SHA %v for repo %v", sha, input.Name) 697 } 698 699 return nil 700 } 701 702 // Run user code and return the combined output of stdout and stderr. 703 func (d *driver) RunUserCode( 704 logger logs.TaggedLogger, 705 environ []string, 706 procStats *pps.ProcessStats, 707 rawDatumTimeout *types.Duration, 708 ) (retErr error) { 709 ctx := d.pachClient.Ctx() 710 d.reportUserCodeStats(logger) 711 defer func(start time.Time) { d.reportDeferredUserCodeStats(retErr, start, procStats, logger) }(time.Now()) 712 logger.Logf("beginning to run user code") 713 defer func(start time.Time) { 714 if retErr != nil { 715 logger.Logf("errored running user code after %v: %v", time.Since(start), retErr) 716 } else { 717 logger.Logf("finished running user code after %v", time.Since(start)) 718 } 719 }(time.Now()) 720 if rawDatumTimeout != nil { 721 datumTimeout, err := types.DurationFromProto(rawDatumTimeout) 722 if err != nil { 723 return errors.EnsureStack(err) 724 } 725 datumTimeoutCtx, cancel := context.WithTimeout(ctx, datumTimeout) 726 defer cancel() 727 ctx = datumTimeoutCtx 728 } 729 730 if len(d.pipelineInfo.Transform.Cmd) == 0 { 731 return errors.New("invalid pipeline transform, no command specified") 732 } 733 734 // Run user code 735 cmd := exec.CommandContext(ctx, d.pipelineInfo.Transform.Cmd[0], d.pipelineInfo.Transform.Cmd[1:]...) 736 if d.pipelineInfo.Transform.Stdin != nil { 737 cmd.Stdin = strings.NewReader(strings.Join(d.pipelineInfo.Transform.Stdin, "\n") + "\n") 738 } 739 cmd.Stdout = logger.WithUserCode() 740 cmd.Stderr = logger.WithUserCode() 741 cmd.Env = environ 742 if d.uid != nil && d.gid != nil { 743 cmd.SysProcAttr = makeCmdCredentials(*d.uid, *d.gid) 744 } 745 // By default, the dockerfile will determine the working dir for the container, so if we 746 // couldn't read the container config with docker, don't touch it. If the pipeline or 747 // worker config explicitly sets the value, then override the container working dir. 748 if d.pipelineInfo.Transform.WorkingDir != "" || d.rootDir != "/" { 749 cmd.Dir = filepath.Join(d.rootDir, d.pipelineInfo.Transform.WorkingDir) 750 } 751 err := cmd.Start() 752 if err != nil { 753 return errors.EnsureStack(err) 754 } 755 // A context with a deadline will successfully cancel/kill 756 // the running process (minus zombies) 757 state, err := cmd.Process.Wait() 758 if err != nil { 759 return errors.EnsureStack(err) 760 } 761 if common.IsDone(ctx) { 762 if err = ctx.Err(); err != nil { 763 return errors.EnsureStack(err) 764 } 765 } 766 767 // Because of this issue: https://github.com/golang/go/issues/18874 768 // We forked os/exec so that we can call just the part of cmd.Wait() that 769 // happens after blocking on the process. Unfortunately calling 770 // cmd.Process.Wait() then cmd.Wait() will produce an error. So instead we 771 // close the IO using this helper 772 err = cmd.WaitIO(state, err) 773 // We ignore broken pipe errors, these occur very occasionally if a user 774 // specifies Stdin but their process doesn't actually read everything from 775 // Stdin. This is a fairly common thing to do, bash by default ignores 776 // broken pipe errors. 777 if err != nil && !strings.Contains(err.Error(), "broken pipe") { 778 // (if err is an acceptable return code, don't return err) 779 exiterr := &exec.ExitError{} 780 if errors.As(err, &exiterr) { 781 if status, ok := exiterr.Sys().(syscall.WaitStatus); ok { 782 for _, returnCode := range d.pipelineInfo.Transform.AcceptReturnCode { 783 if int(returnCode) == status.ExitStatus() { 784 return nil 785 } 786 } 787 } 788 } 789 return errors.EnsureStack(err) 790 } 791 return nil 792 } 793 794 // Run user error code and return the combined output of stdout and stderr. 795 func (d *driver) RunUserErrorHandlingCode(logger logs.TaggedLogger, environ []string, procStats *pps.ProcessStats, rawDatumTimeout *types.Duration) (retErr error) { 796 ctx := d.pachClient.Ctx() 797 logger.Logf("beginning to run user error handling code") 798 defer func(start time.Time) { 799 if retErr != nil { 800 logger.Logf("errored running user error handling code after %v: %v", time.Since(start), retErr) 801 } else { 802 logger.Logf("finished running user error handling code after %v", time.Since(start)) 803 } 804 }(time.Now()) 805 806 cmd := exec.CommandContext(ctx, d.pipelineInfo.Transform.ErrCmd[0], d.pipelineInfo.Transform.ErrCmd[1:]...) 807 if d.pipelineInfo.Transform.ErrStdin != nil { 808 cmd.Stdin = strings.NewReader(strings.Join(d.pipelineInfo.Transform.ErrStdin, "\n") + "\n") 809 } 810 cmd.Stdout = logger.WithUserCode() 811 cmd.Stderr = logger.WithUserCode() 812 cmd.Env = environ 813 if d.uid != nil && d.gid != nil { 814 cmd.SysProcAttr = makeCmdCredentials(*d.uid, *d.gid) 815 } 816 cmd.Dir = d.pipelineInfo.Transform.WorkingDir 817 err := cmd.Start() 818 if err != nil { 819 return errors.EnsureStack(err) 820 } 821 // A context w a deadline will successfully cancel/kill 822 // the running process (minus zombies) 823 state, err := cmd.Process.Wait() 824 if err != nil { 825 return errors.EnsureStack(err) 826 } 827 if common.IsDone(ctx) { 828 if err = ctx.Err(); err != nil { 829 return errors.EnsureStack(err) 830 } 831 } 832 // Because of this issue: https://github.com/golang/go/issues/18874 833 // We forked os/exec so that we can call just the part of cmd.Wait() that 834 // happens after blocking on the process. Unfortunately calling 835 // cmd.Process.Wait() then cmd.Wait() will produce an error. So instead we 836 // close the IO using this helper 837 err = cmd.WaitIO(state, err) 838 // We ignore broken pipe errors, these occur very occasionally if a user 839 // specifies Stdin but their process doesn't actually read everything from 840 // Stdin. This is a fairly common thing to do, bash by default ignores 841 // broken pipe errors. 842 if err != nil && !strings.Contains(err.Error(), "broken pipe") { 843 // (if err is an acceptable return code, don't return err) 844 exiterr := &exec.ExitError{} 845 if errors.As(err, &exiterr) { 846 if status, ok := exiterr.Sys().(syscall.WaitStatus); ok { 847 for _, returnCode := range d.pipelineInfo.Transform.AcceptReturnCode { 848 if int(returnCode) == status.ExitStatus() { 849 return nil 850 } 851 } 852 } 853 } 854 return errors.EnsureStack(err) 855 } 856 return nil 857 } 858 859 func (d *driver) UpdateJobState(jobID string, state pps.JobState, reason string) error { 860 _, err := d.NewSTM(func(stm col.STM) error { 861 jobPtr := &pps.EtcdJobInfo{} 862 if err := d.Jobs().ReadWrite(stm).Get(jobID, jobPtr); err != nil { 863 return errors.EnsureStack(err) 864 } 865 return errors.EnsureStack(ppsutil.UpdateJobState(d.Pipelines().ReadWrite(stm), d.Jobs().ReadWrite(stm), jobPtr, state, reason)) 866 }) 867 return errors.EnsureStack(err) 868 } 869 870 // DeleteJob is identical to updateJobState, except that jobPtr points to a job 871 // that should be deleted rather than marked failed. Jobs may be deleted if 872 // their output commit is deleted. 873 func (d *driver) DeleteJob(stm col.STM, jobPtr *pps.EtcdJobInfo) error { 874 pipelinePtr := &pps.EtcdPipelineInfo{} 875 if err := d.Pipelines().ReadWrite(stm).Update(jobPtr.Pipeline.Name, pipelinePtr, func() error { 876 if pipelinePtr.JobCounts == nil { 877 pipelinePtr.JobCounts = make(map[int32]int32) 878 } 879 if pipelinePtr.JobCounts[int32(jobPtr.State)] != 0 { 880 pipelinePtr.JobCounts[int32(jobPtr.State)]-- 881 } 882 return nil 883 }); err != nil { 884 return errors.EnsureStack(err) 885 } 886 return errors.EnsureStack(d.Jobs().ReadWrite(stm).Delete(jobPtr.Job.ID)) 887 } 888 889 func (d *driver) updateCounter( 890 stat *prometheus.CounterVec, 891 logger logs.TaggedLogger, 892 state string, 893 cb func(prometheus.Counter), 894 ) { 895 labels := []string{d.pipelineInfo.ID, logger.JobID()} 896 if state != "" { 897 labels = append(labels, state) 898 } 899 if counter, err := stat.GetMetricWithLabelValues(labels...); err != nil { 900 logger.Logf("failed to get counter with labels (%v): %v", labels, err) 901 } else { 902 cb(counter) 903 } 904 } 905 906 func (d *driver) updateHistogram( 907 stat *prometheus.HistogramVec, 908 logger logs.TaggedLogger, 909 state string, 910 cb func(prometheus.Observer), 911 ) { 912 labels := []string{d.pipelineInfo.ID, logger.JobID()} 913 if state != "" { 914 labels = append(labels, state) 915 } 916 if hist, err := stat.GetMetricWithLabelValues(labels...); err != nil { 917 logger.Logf("failed to get histogram with labels (%v): %v", labels, err) 918 } else { 919 cb(hist) 920 } 921 } 922 923 func (d *driver) reportUserCodeStats(logger logs.TaggedLogger) { 924 if d.exportStats { 925 d.updateCounter(stats.DatumCount, logger, "started", func(counter prometheus.Counter) { 926 counter.Add(1) 927 }) 928 } 929 } 930 931 func (d *driver) reportDeferredUserCodeStats( 932 err error, 933 start time.Time, 934 procStats *pps.ProcessStats, 935 logger logs.TaggedLogger, 936 ) { 937 duration := time.Since(start) 938 procStats.ProcessTime = types.DurationProto(duration) 939 940 if d.exportStats { 941 state := "errored" 942 if err == nil { 943 state = "finished" 944 } 945 946 d.updateCounter(stats.DatumCount, logger, state, func(counter prometheus.Counter) { 947 counter.Add(1) 948 }) 949 d.updateHistogram(stats.DatumProcTime, logger, state, func(hist prometheus.Observer) { 950 hist.Observe(duration.Seconds()) 951 }) 952 d.updateCounter(stats.DatumProcSecondsCount, logger, "", func(counter prometheus.Counter) { 953 counter.Add(duration.Seconds()) 954 }) 955 } 956 } 957 958 func (d *driver) ReportUploadStats( 959 start time.Time, 960 procStats *pps.ProcessStats, 961 logger logs.TaggedLogger, 962 ) { 963 duration := time.Since(start) 964 procStats.UploadTime = types.DurationProto(duration) 965 966 if d.exportStats { 967 d.updateHistogram(stats.DatumUploadTime, logger, "", func(hist prometheus.Observer) { 968 hist.Observe(duration.Seconds()) 969 }) 970 d.updateCounter(stats.DatumUploadSecondsCount, logger, "", func(counter prometheus.Counter) { 971 counter.Add(duration.Seconds()) 972 }) 973 d.updateHistogram(stats.DatumUploadSize, logger, "", func(hist prometheus.Observer) { 974 hist.Observe(float64(procStats.UploadBytes)) 975 }) 976 d.updateCounter(stats.DatumUploadBytesCount, logger, "", func(counter prometheus.Counter) { 977 counter.Add(float64(procStats.UploadBytes)) 978 }) 979 } 980 } 981 982 func (d *driver) reportDownloadSizeStats( 983 downSize float64, 984 logger logs.TaggedLogger, 985 ) { 986 if d.exportStats { 987 d.updateHistogram(stats.DatumDownloadSize, logger, "", func(hist prometheus.Observer) { 988 hist.Observe(downSize) 989 }) 990 d.updateCounter(stats.DatumDownloadBytesCount, logger, "", func(counter prometheus.Counter) { 991 counter.Add(downSize) 992 }) 993 } 994 } 995 996 func (d *driver) reportDownloadTimeStats( 997 start time.Time, 998 procStats *pps.ProcessStats, 999 logger logs.TaggedLogger, 1000 ) { 1001 duration := time.Since(start) 1002 procStats.DownloadTime = types.DurationProto(duration) 1003 1004 if d.exportStats { 1005 1006 d.updateHistogram(stats.DatumDownloadTime, logger, "", func(hist prometheus.Observer) { 1007 hist.Observe(duration.Seconds()) 1008 }) 1009 d.updateCounter(stats.DatumDownloadSecondsCount, logger, "", func(counter prometheus.Counter) { 1010 counter.Add(duration.Seconds()) 1011 }) 1012 } 1013 } 1014 1015 func (d *driver) unlinkData(inputs []*common.Input) error { 1016 entries, err := ioutil.ReadDir(d.InputDir()) 1017 if err != nil { 1018 return errors.EnsureStack(err) 1019 } 1020 for _, entry := range entries { 1021 if entry.Name() == client.PPSScratchSpace { 1022 continue // don't delete scratch space 1023 } 1024 if err := os.RemoveAll(filepath.Join(d.InputDir(), entry.Name())); err != nil { 1025 return errors.EnsureStack(err) 1026 } 1027 } 1028 return nil 1029 } 1030 1031 func (d *driver) UploadOutput( 1032 dir string, 1033 tag string, 1034 logger logs.TaggedLogger, 1035 inputs []*common.Input, 1036 stats *pps.ProcessStats, 1037 statsTree *hashtree.Ordered, 1038 ) (retBuffer []byte, retErr error) { 1039 defer d.ReportUploadStats(time.Now(), stats, logger) 1040 logger.Logf("starting to upload output") 1041 defer func(start time.Time) { 1042 if retErr != nil { 1043 logger.Logf("errored uploading output after %v: %v", time.Since(start), retErr) 1044 } else { 1045 logger.Logf("finished uploading output after %v", time.Since(start)) 1046 } 1047 }(time.Now()) 1048 1049 // Set up client for writing file data 1050 putObjsClient, err := d.pachClient.ObjectAPIClient.PutObjects(d.pachClient.Ctx()) 1051 if err != nil { 1052 return nil, errors.EnsureStack(err) 1053 } 1054 block := &pfs.Block{Hash: uuid.NewWithoutDashes()} 1055 if err := putObjsClient.Send(&pfs.PutObjectRequest{ 1056 Block: block, 1057 }); err != nil { 1058 return nil, errors.EnsureStack(err) 1059 } 1060 outputPath := filepath.Join(dir, "out") 1061 buf := grpcutil.GetBuffer() 1062 defer grpcutil.PutBuffer(buf) 1063 var offset uint64 1064 var tree *hashtree.Ordered 1065 1066 // Upload all files in output directory 1067 if err := filepath.Walk(outputPath, func(filePath string, info os.FileInfo, err error) error { 1068 if err != nil { 1069 return errors.EnsureStack(err) 1070 } 1071 if !utf8.ValidString(filePath) { 1072 return errors.Errorf("file path is not valid utf-8: %s", filePath) 1073 } 1074 if filePath == outputPath { 1075 tree = hashtree.NewOrdered("/") 1076 return nil 1077 } 1078 relPath, err := filepath.Rel(outputPath, filePath) 1079 if err != nil { 1080 return errors.EnsureStack(err) 1081 } 1082 // Put directory. Even if the directory is empty, that may be useful to 1083 // users 1084 // TODO(msteffen) write a test pipeline that outputs an empty directory and 1085 // make sure it's preserved 1086 if info.IsDir() { 1087 tree.PutDir(relPath) 1088 if statsTree != nil { 1089 statsTree.PutDir(relPath) 1090 } 1091 return nil 1092 } 1093 // Under some circumstances, the user might have copied 1094 // some pipes from the input directory to the output directory. 1095 // Reading from these files will result in job blocking. Thus 1096 // we preemptively detect if the file is a named pipe. 1097 if (info.Mode() & os.ModeNamedPipe) > 0 { 1098 logger.Logf("cannot upload named pipe: %v", relPath) 1099 return errors.EnsureStack(errSpecialFile) 1100 } 1101 // If the output file is a symlink to an input file, we can skip 1102 // the uploading. 1103 if (info.Mode() & os.ModeSymlink) > 0 { 1104 realPath, err := os.Readlink(filePath) 1105 if err != nil { 1106 return errors.EnsureStack(err) 1107 } 1108 1109 // We can only skip the upload if the real path is 1110 // under /pfs, meaning that it's a file that already 1111 // exists in PFS. 1112 if strings.HasPrefix(realPath, d.InputDir()) { 1113 if pathWithInput, err := filepath.Rel(dir, realPath); err == nil { 1114 // The name of the input 1115 inputName := strings.Split(pathWithInput, string(os.PathSeparator))[0] 1116 var input *common.Input 1117 for _, i := range inputs { 1118 if i.Name == inputName { 1119 input = i 1120 } 1121 } 1122 // this changes realPath from `/pfs/input/...` to `/scratch/<id>/input/...` 1123 realPath = filepath.Join(dir, pathWithInput) 1124 if input != nil { 1125 return filepath.Walk(realPath, func(filePath string, info os.FileInfo, err error) error { 1126 if err != nil { 1127 return errors.EnsureStack(err) 1128 } 1129 rel, err := filepath.Rel(realPath, filePath) 1130 if err != nil { 1131 return errors.EnsureStack(err) 1132 } 1133 subRelPath := filepath.Join(relPath, rel) 1134 // The path of the input file 1135 pfsPath, err := filepath.Rel(filepath.Join(dir, input.Name), filePath) 1136 if err != nil { 1137 return errors.EnsureStack(err) 1138 } 1139 if info.IsDir() { 1140 tree.PutDir(subRelPath) 1141 if statsTree != nil { 1142 statsTree.PutDir(subRelPath) 1143 } 1144 return nil 1145 } 1146 fc := input.FileInfo.File.Commit 1147 fileInfo, err := d.pachClient.InspectFile(fc.Repo.Name, fc.ID, pfsPath) 1148 if err != nil { 1149 return errors.EnsureStack(err) 1150 } 1151 var blockRefs []*pfs.BlockRef 1152 for _, object := range fileInfo.Objects { 1153 objectInfo, err := d.pachClient.InspectObject(object.Hash) 1154 if err != nil { 1155 return errors.EnsureStack(err) 1156 } 1157 blockRefs = append(blockRefs, objectInfo.BlockRef) 1158 } 1159 blockRefs = append(blockRefs, fileInfo.BlockRefs...) 1160 n := &hashtree.FileNodeProto{BlockRefs: blockRefs} 1161 tree.PutFile(subRelPath, fileInfo.Hash, int64(fileInfo.SizeBytes), n) 1162 if statsTree != nil { 1163 statsTree.PutFile(subRelPath, fileInfo.Hash, int64(fileInfo.SizeBytes), n) 1164 } 1165 return nil 1166 }) 1167 } 1168 } 1169 } 1170 } 1171 // Open local file that is being uploaded 1172 f, err := os.Open(filePath) 1173 if err != nil { 1174 // if the error is that the spout marker file is missing, that's fine, just skip to the next file 1175 if d.PipelineInfo().Spout != nil { 1176 if strings.Contains(err.Error(), filepath.Join("out", d.PipelineInfo().Spout.Marker)) { 1177 return nil 1178 } 1179 } 1180 return errors.Wrapf(err, "os.Open(%s)", filePath) 1181 } 1182 defer func() { 1183 if err := f.Close(); err != nil && retErr == nil { 1184 retErr = err 1185 } 1186 }() 1187 var size int64 1188 h := pfs.NewHash() 1189 r := io.TeeReader(f, h) 1190 // Write local file to object storage block 1191 for { 1192 n, err := r.Read(buf) 1193 if n == 0 && err != nil { 1194 if errors.Is(err, io.EOF) { 1195 break 1196 } 1197 return errors.EnsureStack(err) 1198 } 1199 if err := putObjsClient.Send(&pfs.PutObjectRequest{ 1200 Value: buf[:n], 1201 }); err != nil { 1202 return errors.EnsureStack(err) 1203 } 1204 size += int64(n) 1205 } 1206 n := &hashtree.FileNodeProto{ 1207 BlockRefs: []*pfs.BlockRef{ 1208 &pfs.BlockRef{ 1209 Block: block, 1210 Range: &pfs.ByteRange{ 1211 Lower: offset, 1212 Upper: offset + uint64(size), 1213 }, 1214 }, 1215 }, 1216 } 1217 hash := h.Sum(nil) 1218 tree.PutFile(relPath, hash, size, n) 1219 if statsTree != nil { 1220 statsTree.PutFile(relPath, hash, size, n) 1221 } 1222 offset += uint64(size) 1223 stats.UploadBytes += uint64(size) 1224 return nil 1225 }); err != nil { 1226 return nil, errors.Wrap(err, "error walking output") 1227 } 1228 if _, err := putObjsClient.CloseAndRecv(); err != nil && !errors.Is(err, io.EOF) { 1229 return nil, errors.EnsureStack(err) 1230 } 1231 // Serialize datum hashtree 1232 b := &bytes.Buffer{} 1233 if err := tree.Serialize(b); err != nil { 1234 return nil, err 1235 } 1236 // Write datum hashtree to object storage 1237 w, err := d.pachClient.PutObjectAsync([]*pfs.Tag{client.NewTag(tag)}) 1238 if err != nil { 1239 return nil, errors.EnsureStack(err) 1240 } 1241 defer func() { 1242 if err := w.Close(); err != nil && retErr != nil { 1243 retErr = errors.EnsureStack(err) 1244 } 1245 }() 1246 if _, err := w.Write(b.Bytes()); err != nil { 1247 return nil, errors.EnsureStack(err) 1248 } 1249 return b.Bytes(), nil 1250 } 1251 1252 func (d *driver) UserCodeEnv( 1253 jobID string, 1254 outputCommit *pfs.Commit, 1255 inputs []*common.Input, 1256 ) []string { 1257 result := os.Environ() 1258 1259 for _, input := range inputs { 1260 result = append(result, fmt.Sprintf("%s=%s", input.Name, filepath.Join(d.InputDir(), input.Name, input.FileInfo.File.Path))) 1261 result = append(result, fmt.Sprintf("%s_COMMIT=%s", input.Name, input.FileInfo.File.Commit.ID)) 1262 } 1263 1264 if jobID != "" { 1265 result = append(result, fmt.Sprintf("%s=%s", client.JobIDEnv, jobID)) 1266 if ppsutil.ContainsS3Inputs(d.PipelineInfo().Input) || d.PipelineInfo().S3Out { 1267 // TODO(msteffen) Instead of reading S3GATEWAY_PORT directly, worker/main.go 1268 // should pass its ServiceEnv to worker.NewAPIServer, which should store it 1269 // in 'a'. However, requiring worker.APIServer to have a ServiceEnv would 1270 // break the worker.APIServer initialization in newTestAPIServer (in 1271 // worker/worker_test.go), which uses mock clients but has no good way to 1272 // mock a ServiceEnv. Once we can create mock ServiceEnvs, we should store 1273 // a ServiceEnv in worker.APIServer, rewrite newTestAPIServer and 1274 // NewAPIServer, and then change this code. 1275 result = append( 1276 result, 1277 fmt.Sprintf("S3_ENDPOINT=http://%s.%s:%s", 1278 ppsutil.SidecarS3GatewayService(jobID), 1279 d.Namespace(), 1280 os.Getenv("S3GATEWAY_PORT"), 1281 ), 1282 ) 1283 } 1284 } 1285 1286 if outputCommit != nil { 1287 result = append(result, fmt.Sprintf("%s=%s", client.OutputCommitIDEnv, outputCommit.ID)) 1288 } 1289 1290 return result 1291 } 1292 1293 func (d *driver) Egress(commit *pfs.Commit, egressURL string) error { 1294 // copy the pach client (preserving auth info) so we can set a different 1295 // number of concurrent streams 1296 pachClient := d.PachClient().WithCtx(d.PachClient().Ctx()) 1297 pachClient.SetMaxConcurrentStreams(100) 1298 1299 url, err := obj.ParseURL(egressURL) 1300 if err != nil { 1301 return err 1302 } 1303 objClient, err := obj.NewClientFromURLAndSecret(url, false) 1304 if err != nil { 1305 return err 1306 } 1307 return filesync.PushObj(pachClient, commit, objClient, url.Object) 1308 }