You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/04/10 08:25:00 UTC

[GitHub] [arrow] kszucs commented on a change in pull request #9913: ARROW-12263: [Dev][Packaging] Move Crossbow to Archery

kszucs commented on a change in pull request #9913:
URL: https://github.com/apache/arrow/pull/9913#discussion_r611015829



##########
File path: dev/archery/archery/bot.py
##########
@@ -248,81 +167,94 @@ def actions(ctx):
               help='Crossbow repository on github to use')
 @click.pass_obj
 def crossbow(obj, crossbow):
-    """Trigger crossbow builds for this pull request"""
+    """
+    Trigger crossbow builds for this pull request
+    """
     obj['crossbow_repo'] = crossbow
 
 
+def _clone_arrow_and_crossbow(dest, crossbow_repo, pull_request):
+    """
+    Clone the repositories and initialize crossbow objects.
+
+    Parameters
+    ----------
+    dest : Path
+        Filesystem path to clone the repositories to.
+    crossbow_repo : str
+        Github repository name, like kszucs/crossbow.
+    pull_request : pygithub.PullRequest
+        Object containing information about the pull request the comment bot
+        was triggered from.
+    """
+    arrow_path = dest / 'arrow'
+    queue_path = dest / 'crossbow'
+
+    # clone arrow and checkout the pull request's branch
+    pull_request_ref = 'pull/{}/head:{}'.format(
+        pull_request.number, pull_request.head.ref
+    )
+    git.clone(pull_request.base.repo.clone_url, str(arrow_path))
+    git.fetch('origin', pull_request_ref, git_dir=arrow_path)
+    git.checkout(pull_request.head.ref, git_dir=arrow_path)
+
+    # clone crossbow repository
+    crossbow_url = 'https://github.com/{}'.format(crossbow_repo)
+    git.clone(crossbow_url, str(queue_path))
+
+    # initialize crossbow objects
+    github_token = os.environ['CROSSBOW_GITHUB_TOKEN']
+    arrow = Repo(arrow_path)
+    queue = Queue(queue_path, github_token=github_token, require_https=True)
+
+    return (arrow, queue)
+
+
 @crossbow.command()
 @click.argument('tasks', nargs=-1, required=False)
 @click.option('--group', '-g', 'groups', multiple=True,
               help='Submit task groups as defined in tests.yml')
 @click.option('--param', '-p', 'params', multiple=True,
               help='Additional task parameters for rendering the CI templates')
-@click.option('--dry-run/--push', default=False,
-              help='Just display the new changelog, don\'t write it')
+@click.option('--arrow-version', '-v', default=None,
+              help='Set target version explicitly.')
 @click.pass_obj
-def submit(obj, tasks, groups, params, dry_run):
-    """Submit crossbow testing tasks.
+def submit(obj, tasks, groups, params, arrow_version):
+    """
+    Submit crossbow testing tasks.
 
     See groups defined in arrow/dev/tasks/tests.yml
     """
-    from ruamel.yaml import YAML
-
-    git = Git()
-
-    # construct crossbow arguments
-    args = []
-    if dry_run:
-        args.append('--dry-run')
-
-    for p in params:
-        args.extend(['-p', p])
-    for g in groups:
-        args.extend(['-g', g])
-    for t in tasks:
-        args.append(t)
-
-    # pygithub pull request object
-    pr = obj['pull']
-    crossbow_url = 'https://github.com/{}'.format(obj['crossbow_repo'])
-
+    crossbow_repo = obj['crossbow_repo']
+    pull_request = obj['pull_request']
     with tempfile.TemporaryDirectory() as tmpdir:
         tmpdir = Path(tmpdir)
-        arrow = tmpdir / 'arrow'
-        queue = tmpdir / 'crossbow'
-
-        # clone arrow and checkout the pull request's branch
-        git.clone(pr.base.repo.clone_url, str(arrow))
-        git.fetch('origin', 'pull/{}/head:{}'.format(pr.number, pr.head.ref),
-                  git_dir=str(arrow))
-        git.checkout(pr.head.ref, git_dir=str(arrow))
-
-        # clone crossbow
-        git.clone(crossbow_url, str(queue))
-
-        # submit the crossbow tasks
-        result = Path('result.yml').resolve()
-        xbow = Crossbow(str(arrow / 'dev' / 'tasks' / 'crossbow.py'))
-        xbow.run(
-            '--queue-path', str(queue),
-            '--output-file', str(result),
-            'submit',
-            '--job-prefix', 'actions',
-            # don't rely on crossbow's remote and branch detection, because
-            # it doesn't work without a tracking upstream branch
-            '--arrow-remote', pr.head.repo.clone_url,
-            '--arrow-branch', pr.head.ref,
-            *args
+        arrow, queue = _clone_arrow_and_crossbow(
+            dest=Path(tmpdir),
+            crossbow_repo=crossbow_repo,
+            pull_request=pull_request,
         )
+        # load available tasks configuration and groups from yaml
+        config = Config.load_yaml(arrow.path / "dev" / "tasks" / "tasks.yml")
+        config.validate()
+
+        # initialize the crossbow build's target repository
+        target = Target.from_repo(arrow, version=arrow_version,
+                                  remote=pull_request.base.repo.clone_url)

Review comment:
       We should, sorry about that! The fix is at https://github.com/apache/arrow/pull/9978




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org