file.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584
  1. package ops
  2. import (
  3. "bytes"
  4. "context"
  5. "encoding/json"
  6. "fmt"
  7. "path"
  8. "runtime"
  9. "sort"
  10. "sync"
  11. "github.com/moby/buildkit/cache"
  12. "github.com/moby/buildkit/cache/metadata"
  13. "github.com/moby/buildkit/session"
  14. "github.com/moby/buildkit/solver"
  15. "github.com/moby/buildkit/solver/llbsolver"
  16. "github.com/moby/buildkit/solver/llbsolver/file"
  17. "github.com/moby/buildkit/solver/llbsolver/ops/fileoptypes"
  18. "github.com/moby/buildkit/solver/pb"
  19. "github.com/moby/buildkit/util/flightcontrol"
  20. "github.com/moby/buildkit/worker"
  21. digest "github.com/opencontainers/go-digest"
  22. "github.com/pkg/errors"
  23. "golang.org/x/sync/errgroup"
  24. )
  25. const fileCacheType = "buildkit.file.v0"
  26. type fileOp struct {
  27. op *pb.FileOp
  28. md *metadata.Store
  29. w worker.Worker
  30. solver *FileOpSolver
  31. numInputs int
  32. }
  33. func NewFileOp(v solver.Vertex, op *pb.Op_File, cm cache.Manager, md *metadata.Store, w worker.Worker) (solver.Op, error) {
  34. if err := llbsolver.ValidateOp(&pb.Op{Op: op}); err != nil {
  35. return nil, err
  36. }
  37. return &fileOp{
  38. op: op.File,
  39. md: md,
  40. numInputs: len(v.Inputs()),
  41. w: w,
  42. solver: NewFileOpSolver(&file.Backend{}, file.NewRefManager(cm)),
  43. }, nil
  44. }
  45. func (f *fileOp) CacheMap(ctx context.Context, g session.Group, index int) (*solver.CacheMap, bool, error) {
  46. selectors := map[int]map[llbsolver.Selector]struct{}{}
  47. invalidSelectors := map[int]struct{}{}
  48. actions := make([][]byte, 0, len(f.op.Actions))
  49. markInvalid := func(idx pb.InputIndex) {
  50. if idx != -1 {
  51. invalidSelectors[int(idx)] = struct{}{}
  52. }
  53. }
  54. for _, action := range f.op.Actions {
  55. var dt []byte
  56. var err error
  57. switch a := action.Action.(type) {
  58. case *pb.FileAction_Mkdir:
  59. p := *a.Mkdir
  60. markInvalid(action.Input)
  61. processOwner(p.Owner, selectors)
  62. dt, err = json.Marshal(p)
  63. if err != nil {
  64. return nil, false, err
  65. }
  66. case *pb.FileAction_Mkfile:
  67. p := *a.Mkfile
  68. markInvalid(action.Input)
  69. processOwner(p.Owner, selectors)
  70. dt, err = json.Marshal(p)
  71. if err != nil {
  72. return nil, false, err
  73. }
  74. case *pb.FileAction_Rm:
  75. p := *a.Rm
  76. markInvalid(action.Input)
  77. dt, err = json.Marshal(p)
  78. if err != nil {
  79. return nil, false, err
  80. }
  81. case *pb.FileAction_Copy:
  82. p := *a.Copy
  83. markInvalid(action.Input)
  84. processOwner(p.Owner, selectors)
  85. if action.SecondaryInput != -1 && int(action.SecondaryInput) < f.numInputs {
  86. addSelector(selectors, int(action.SecondaryInput), p.Src, p.AllowWildcard, p.FollowSymlink)
  87. p.Src = path.Base(p.Src)
  88. }
  89. dt, err = json.Marshal(p)
  90. if err != nil {
  91. return nil, false, err
  92. }
  93. }
  94. actions = append(actions, dt)
  95. }
  96. dt, err := json.Marshal(struct {
  97. Type string
  98. Actions [][]byte
  99. }{
  100. Type: fileCacheType,
  101. Actions: actions,
  102. })
  103. if err != nil {
  104. return nil, false, err
  105. }
  106. cm := &solver.CacheMap{
  107. Digest: digest.FromBytes(dt),
  108. Deps: make([]struct {
  109. Selector digest.Digest
  110. ComputeDigestFunc solver.ResultBasedCacheFunc
  111. }, f.numInputs),
  112. }
  113. for idx, m := range selectors {
  114. if _, ok := invalidSelectors[idx]; ok {
  115. continue
  116. }
  117. dgsts := make([][]byte, 0, len(m))
  118. for k := range m {
  119. dgsts = append(dgsts, []byte(k.Path))
  120. }
  121. sort.Slice(dgsts, func(i, j int) bool {
  122. return bytes.Compare(dgsts[i], dgsts[j]) > 0
  123. })
  124. cm.Deps[idx].Selector = digest.FromBytes(bytes.Join(dgsts, []byte{0}))
  125. cm.Deps[idx].ComputeDigestFunc = llbsolver.NewContentHashFunc(dedupeSelectors(m))
  126. }
  127. return cm, true, nil
  128. }
  129. func (f *fileOp) Exec(ctx context.Context, g session.Group, inputs []solver.Result) ([]solver.Result, error) {
  130. inpRefs := make([]fileoptypes.Ref, 0, len(inputs))
  131. for _, inp := range inputs {
  132. workerRef, ok := inp.Sys().(*worker.WorkerRef)
  133. if !ok {
  134. return nil, errors.Errorf("invalid reference for exec %T", inp.Sys())
  135. }
  136. inpRefs = append(inpRefs, workerRef.ImmutableRef)
  137. }
  138. outs, err := f.solver.Solve(ctx, inpRefs, f.op.Actions)
  139. if err != nil {
  140. return nil, err
  141. }
  142. outResults := make([]solver.Result, 0, len(outs))
  143. for _, out := range outs {
  144. outResults = append(outResults, worker.NewWorkerRefResult(out.(cache.ImmutableRef), f.w))
  145. }
  146. return outResults, nil
  147. }
  148. func addSelector(m map[int]map[llbsolver.Selector]struct{}, idx int, sel string, wildcard, followLinks bool) {
  149. mm, ok := m[idx]
  150. if !ok {
  151. mm = map[llbsolver.Selector]struct{}{}
  152. m[idx] = mm
  153. }
  154. s := llbsolver.Selector{Path: sel}
  155. if wildcard && containsWildcards(sel) {
  156. s.Wildcard = true
  157. }
  158. if followLinks {
  159. s.FollowLinks = true
  160. }
  161. mm[s] = struct{}{}
  162. }
  163. func containsWildcards(name string) bool {
  164. isWindows := runtime.GOOS == "windows"
  165. for i := 0; i < len(name); i++ {
  166. ch := name[i]
  167. if ch == '\\' && !isWindows {
  168. i++
  169. } else if ch == '*' || ch == '?' || ch == '[' {
  170. return true
  171. }
  172. }
  173. return false
  174. }
  175. func dedupeSelectors(m map[llbsolver.Selector]struct{}) []llbsolver.Selector {
  176. paths := make([]string, 0, len(m))
  177. pathsFollow := make([]string, 0, len(m))
  178. for sel := range m {
  179. if !sel.Wildcard {
  180. if sel.FollowLinks {
  181. pathsFollow = append(pathsFollow, sel.Path)
  182. } else {
  183. paths = append(paths, sel.Path)
  184. }
  185. }
  186. }
  187. paths = dedupePaths(paths)
  188. pathsFollow = dedupePaths(pathsFollow)
  189. selectors := make([]llbsolver.Selector, 0, len(m))
  190. for _, p := range paths {
  191. selectors = append(selectors, llbsolver.Selector{Path: p})
  192. }
  193. for _, p := range pathsFollow {
  194. selectors = append(selectors, llbsolver.Selector{Path: p, FollowLinks: true})
  195. }
  196. for sel := range m {
  197. if sel.Wildcard {
  198. selectors = append(selectors, sel)
  199. }
  200. }
  201. sort.Slice(selectors, func(i, j int) bool {
  202. return selectors[i].Path < selectors[j].Path
  203. })
  204. return selectors
  205. }
  206. func processOwner(chopt *pb.ChownOpt, selectors map[int]map[llbsolver.Selector]struct{}) error {
  207. if chopt == nil {
  208. return nil
  209. }
  210. if chopt.User != nil {
  211. if u, ok := chopt.User.User.(*pb.UserOpt_ByName); ok {
  212. if u.ByName.Input < 0 {
  213. return errors.Errorf("invalid user index %d", u.ByName.Input)
  214. }
  215. addSelector(selectors, int(u.ByName.Input), "/etc/passwd", false, true)
  216. }
  217. }
  218. if chopt.Group != nil {
  219. if u, ok := chopt.Group.User.(*pb.UserOpt_ByName); ok {
  220. if u.ByName.Input < 0 {
  221. return errors.Errorf("invalid user index %d", u.ByName.Input)
  222. }
  223. addSelector(selectors, int(u.ByName.Input), "/etc/group", false, true)
  224. }
  225. }
  226. return nil
  227. }
  228. func NewFileOpSolver(b fileoptypes.Backend, r fileoptypes.RefManager) *FileOpSolver {
  229. return &FileOpSolver{
  230. b: b,
  231. r: r,
  232. outs: map[int]int{},
  233. ins: map[int]input{},
  234. }
  235. }
  236. type FileOpSolver struct {
  237. b fileoptypes.Backend
  238. r fileoptypes.RefManager
  239. mu sync.Mutex
  240. outs map[int]int
  241. ins map[int]input
  242. g flightcontrol.Group
  243. }
  244. type input struct {
  245. requiresCommit bool
  246. mount fileoptypes.Mount
  247. ref fileoptypes.Ref
  248. }
  249. func (s *FileOpSolver) Solve(ctx context.Context, inputs []fileoptypes.Ref, actions []*pb.FileAction) ([]fileoptypes.Ref, error) {
  250. for i, a := range actions {
  251. if int(a.Input) < -1 || int(a.Input) >= len(inputs)+len(actions) {
  252. return nil, errors.Errorf("invalid input index %d, %d provided", a.Input, len(inputs)+len(actions))
  253. }
  254. if int(a.SecondaryInput) < -1 || int(a.SecondaryInput) >= len(inputs)+len(actions) {
  255. return nil, errors.Errorf("invalid secondary input index %d, %d provided", a.Input, len(inputs))
  256. }
  257. inp, ok := s.ins[int(a.Input)]
  258. if ok {
  259. inp.requiresCommit = true
  260. }
  261. s.ins[int(a.Input)] = inp
  262. inp, ok = s.ins[int(a.SecondaryInput)]
  263. if ok {
  264. inp.requiresCommit = true
  265. }
  266. s.ins[int(a.SecondaryInput)] = inp
  267. if a.Output != -1 {
  268. if _, ok := s.outs[int(a.Output)]; ok {
  269. return nil, errors.Errorf("duplicate output %d", a.Output)
  270. }
  271. idx := len(inputs) + i
  272. s.outs[int(a.Output)] = idx
  273. s.ins[idx] = input{requiresCommit: true}
  274. }
  275. }
  276. if len(s.outs) == 0 {
  277. return nil, errors.Errorf("no outputs specified")
  278. }
  279. for i := 0; i < len(s.outs); i++ {
  280. if _, ok := s.outs[i]; !ok {
  281. return nil, errors.Errorf("missing output index %d", i)
  282. }
  283. }
  284. defer func() {
  285. for _, in := range s.ins {
  286. if in.ref == nil && in.mount != nil {
  287. in.mount.Release(context.TODO())
  288. }
  289. }
  290. }()
  291. outs := make([]fileoptypes.Ref, len(s.outs))
  292. eg, ctx := errgroup.WithContext(ctx)
  293. for i, idx := range s.outs {
  294. func(i, idx int) {
  295. eg.Go(func() error {
  296. if err := s.validate(idx, inputs, actions, nil); err != nil {
  297. return err
  298. }
  299. inp, err := s.getInput(ctx, idx, inputs, actions)
  300. if err != nil {
  301. return err
  302. }
  303. outs[i] = inp.ref
  304. return nil
  305. })
  306. }(i, idx)
  307. }
  308. if err := eg.Wait(); err != nil {
  309. for _, r := range outs {
  310. if r != nil {
  311. r.Release(context.TODO())
  312. }
  313. }
  314. return nil, err
  315. }
  316. return outs, nil
  317. }
  318. func (s *FileOpSolver) validate(idx int, inputs []fileoptypes.Ref, actions []*pb.FileAction, loaded []int) error {
  319. for _, check := range loaded {
  320. if idx == check {
  321. return errors.Errorf("loop from index %d", idx)
  322. }
  323. }
  324. if idx < len(inputs) {
  325. return nil
  326. }
  327. loaded = append(loaded, idx)
  328. action := actions[idx-len(inputs)]
  329. for _, inp := range []int{int(action.Input), int(action.SecondaryInput)} {
  330. if err := s.validate(inp, inputs, actions, loaded); err != nil {
  331. return err
  332. }
  333. }
  334. return nil
  335. }
  336. func (s *FileOpSolver) getInput(ctx context.Context, idx int, inputs []fileoptypes.Ref, actions []*pb.FileAction) (input, error) {
  337. inp, err := s.g.Do(ctx, fmt.Sprintf("inp-%d", idx), func(ctx context.Context) (_ interface{}, err error) {
  338. s.mu.Lock()
  339. inp := s.ins[idx]
  340. s.mu.Unlock()
  341. if inp.mount != nil || inp.ref != nil {
  342. return inp, nil
  343. }
  344. if idx < len(inputs) {
  345. inp.ref = inputs[idx]
  346. s.mu.Lock()
  347. s.ins[idx] = inp
  348. s.mu.Unlock()
  349. return inp, nil
  350. }
  351. var inpMount, inpMountSecondary fileoptypes.Mount
  352. var toRelease []fileoptypes.Mount
  353. var inpMountPrepared bool
  354. defer func() {
  355. for _, m := range toRelease {
  356. m.Release(context.TODO())
  357. }
  358. if err != nil && inpMount != nil && inpMountPrepared {
  359. inpMount.Release(context.TODO())
  360. }
  361. }()
  362. action := actions[idx-len(inputs)]
  363. loadInput := func(ctx context.Context) func() error {
  364. return func() error {
  365. inp, err := s.getInput(ctx, int(action.Input), inputs, actions)
  366. if err != nil {
  367. return err
  368. }
  369. if inp.ref != nil {
  370. m, err := s.r.Prepare(ctx, inp.ref, false)
  371. if err != nil {
  372. return err
  373. }
  374. inpMount = m
  375. inpMountPrepared = true
  376. return nil
  377. }
  378. inpMount = inp.mount
  379. return nil
  380. }
  381. }
  382. loadSecondaryInput := func(ctx context.Context) func() error {
  383. return func() error {
  384. inp, err := s.getInput(ctx, int(action.SecondaryInput), inputs, actions)
  385. if err != nil {
  386. return err
  387. }
  388. if inp.ref != nil {
  389. m, err := s.r.Prepare(ctx, inp.ref, true)
  390. if err != nil {
  391. return err
  392. }
  393. inpMountSecondary = m
  394. toRelease = append(toRelease, m)
  395. return nil
  396. }
  397. inpMountSecondary = inp.mount
  398. return nil
  399. }
  400. }
  401. loadUser := func(ctx context.Context, uopt *pb.UserOpt) (fileoptypes.Mount, error) {
  402. if uopt == nil {
  403. return nil, nil
  404. }
  405. switch u := uopt.User.(type) {
  406. case *pb.UserOpt_ByName:
  407. var m fileoptypes.Mount
  408. if u.ByName.Input < 0 {
  409. return nil, errors.Errorf("invalid user index: %d", u.ByName.Input)
  410. }
  411. inp, err := s.getInput(ctx, int(u.ByName.Input), inputs, actions)
  412. if err != nil {
  413. return nil, err
  414. }
  415. if inp.ref != nil {
  416. mm, err := s.r.Prepare(ctx, inp.ref, true)
  417. if err != nil {
  418. return nil, err
  419. }
  420. toRelease = append(toRelease, mm)
  421. m = mm
  422. } else {
  423. m = inp.mount
  424. }
  425. return m, nil
  426. default:
  427. return nil, nil
  428. }
  429. }
  430. loadOwner := func(ctx context.Context, chopt *pb.ChownOpt) (fileoptypes.Mount, fileoptypes.Mount, error) {
  431. if chopt == nil {
  432. return nil, nil, nil
  433. }
  434. um, err := loadUser(ctx, chopt.User)
  435. if err != nil {
  436. return nil, nil, err
  437. }
  438. gm, err := loadUser(ctx, chopt.Group)
  439. if err != nil {
  440. return nil, nil, err
  441. }
  442. return um, gm, nil
  443. }
  444. if action.Input != -1 && action.SecondaryInput != -1 {
  445. eg, ctx := errgroup.WithContext(ctx)
  446. eg.Go(loadInput(ctx))
  447. eg.Go(loadSecondaryInput(ctx))
  448. if err := eg.Wait(); err != nil {
  449. return nil, err
  450. }
  451. } else {
  452. if action.Input != -1 {
  453. if err := loadInput(ctx)(); err != nil {
  454. return nil, err
  455. }
  456. }
  457. if action.SecondaryInput != -1 {
  458. if err := loadSecondaryInput(ctx)(); err != nil {
  459. return nil, err
  460. }
  461. }
  462. }
  463. if inpMount == nil {
  464. m, err := s.r.Prepare(ctx, nil, false)
  465. if err != nil {
  466. return nil, err
  467. }
  468. inpMount = m
  469. inpMountPrepared = true
  470. }
  471. switch a := action.Action.(type) {
  472. case *pb.FileAction_Mkdir:
  473. user, group, err := loadOwner(ctx, a.Mkdir.Owner)
  474. if err != nil {
  475. return nil, err
  476. }
  477. if err := s.b.Mkdir(ctx, inpMount, user, group, *a.Mkdir); err != nil {
  478. return nil, err
  479. }
  480. case *pb.FileAction_Mkfile:
  481. user, group, err := loadOwner(ctx, a.Mkfile.Owner)
  482. if err != nil {
  483. return nil, err
  484. }
  485. if err := s.b.Mkfile(ctx, inpMount, user, group, *a.Mkfile); err != nil {
  486. return nil, err
  487. }
  488. case *pb.FileAction_Rm:
  489. if err := s.b.Rm(ctx, inpMount, *a.Rm); err != nil {
  490. return nil, err
  491. }
  492. case *pb.FileAction_Copy:
  493. if inpMountSecondary == nil {
  494. m, err := s.r.Prepare(ctx, nil, true)
  495. if err != nil {
  496. return nil, err
  497. }
  498. inpMountSecondary = m
  499. }
  500. user, group, err := loadOwner(ctx, a.Copy.Owner)
  501. if err != nil {
  502. return nil, err
  503. }
  504. if err := s.b.Copy(ctx, inpMountSecondary, inpMount, user, group, *a.Copy); err != nil {
  505. return nil, err
  506. }
  507. default:
  508. return nil, errors.Errorf("invalid action type %T", action.Action)
  509. }
  510. if inp.requiresCommit {
  511. ref, err := s.r.Commit(ctx, inpMount)
  512. if err != nil {
  513. return nil, err
  514. }
  515. inp.ref = ref
  516. } else {
  517. inp.mount = inpMount
  518. }
  519. s.mu.Lock()
  520. s.ins[idx] = inp
  521. s.mu.Unlock()
  522. return inp, nil
  523. })
  524. if err != nil {
  525. return input{}, err
  526. }
  527. return inp.(input), err
  528. }