You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ks...@apache.org on 2020/05/14 10:34:12 UTC

[arrow] 01/03: ARROW-8702: [Packaging][C#] Build NuGet packages in release process

This is an automated email from the ASF dual-hosted git repository.

kszucs pushed a commit to branch maint-0.17.x
in repository https://gitbox.apache.org/repos/asf/arrow.git

commit bc24f2be582ef7978e0615cca4eb59f164583545
Author: Sutou Kouhei <ko...@clear-code.com>
AuthorDate: Thu May 7 05:27:27 2020 +0900

    ARROW-8702: [Packaging][C#] Build NuGet packages in release process
    
    We released NuGet packages as unofficial packages because we only vote
    Apache Arrow C# source code. We didn't vote NuGet packages.
    
    We can vote NuGet packages with this change. We can mark NuGet
    packages as official packages in the next release.
    
    Closes #7104 from kou/packaging-nuget-crossbow
    
    Authored-by: Sutou Kouhei <ko...@clear-code.com>
    Signed-off-by: Sutou Kouhei <ko...@clear-code.com>
---
 .github/workflows/csharp.yml                  |   8 ++
 ci/docker/ubuntu-18.04-csharp.dockerfile      |   2 +
 ci/scripts/{csharp_test.sh => csharp_pack.sh} |   4 +-
 ci/scripts/csharp_test.sh                     |   3 +
 dev/release/03-binary.sh                      |   4 +
 dev/release/binary-task.rb                    | 183 +++++++++++++++++---------
 dev/release/download_rc_binaries.py           |   4 +-
 dev/release/post-02-binary.sh                 |   4 +
 dev/release/post-06-csharp.sh                 |  37 +++---
 dev/tasks/crossbow.py                         |  16 ++-
 dev/tasks/nuget-packages/github.linux.yml     |  82 ++++++++++++
 dev/tasks/tasks.yml                           |  16 ++-
 docker-compose.yml                            |   7 +-
 13 files changed, 275 insertions(+), 95 deletions(-)

diff --git a/.github/workflows/csharp.yml b/.github/workflows/csharp.yml
index 4a64c09..4cc00c3 100644
--- a/.github/workflows/csharp.yml
+++ b/.github/workflows/csharp.yml
@@ -51,6 +51,9 @@ jobs:
       - name: Fetch Submodules and Tags
         shell: bash
         run: ci/scripts/util_checkout.sh
+      - name: Install Source Link
+        shell: bash
+        run: dotnet tool install --global sourcelink
       - name: Build
         shell: bash
         run: ci/scripts/csharp_build.sh $(pwd)
@@ -78,6 +81,8 @@ jobs:
       - name: Fetch Submodules and Tags
         shell: bash
         run: ci/scripts/util_checkout.sh
+      - name: Install Source Link
+        run: dotnet tool install --global sourcelink
       - name: Build
         shell: bash
         run: ci/scripts/csharp_build.sh $(pwd)
@@ -102,6 +107,9 @@ jobs:
         uses: actions/checkout@v2
         with:
           fetch-depth: 0
+      - name: Install Source Link
+        shell: bash
+        run: dotnet tool install --global sourcelink
       - name: Fetch Submodules and Tags
         shell: bash
         run: ci/scripts/util_checkout.sh
diff --git a/ci/docker/ubuntu-18.04-csharp.dockerfile b/ci/docker/ubuntu-18.04-csharp.dockerfile
index d602958..237ce6a 100644
--- a/ci/docker/ubuntu-18.04-csharp.dockerfile
+++ b/ci/docker/ubuntu-18.04-csharp.dockerfile
@@ -18,3 +18,5 @@
 ARG platform=bionic
 ARG dotnet=2.1
 FROM mcr.microsoft.com/dotnet/core/sdk:${dotnet}-${platform}
+
+RUN dotnet tool install --tool-path /usr/local/bin sourcelink
diff --git a/ci/scripts/csharp_test.sh b/ci/scripts/csharp_pack.sh
similarity index 96%
copy from ci/scripts/csharp_test.sh
copy to ci/scripts/csharp_pack.sh
index 37702ce..e9dfc66 100755
--- a/ci/scripts/csharp_test.sh
+++ b/ci/scripts/csharp_pack.sh
@@ -17,10 +17,10 @@
 # specific language governing permissions and limitations
 # under the License.
 
-set -ex
+set -eux
 
 source_dir=${1}/csharp
 
 pushd ${source_dir}
-dotnet test
+dotnet pack -c Release
 popd
diff --git a/ci/scripts/csharp_test.sh b/ci/scripts/csharp_test.sh
index 37702ce..9e4e35d 100755
--- a/ci/scripts/csharp_test.sh
+++ b/ci/scripts/csharp_test.sh
@@ -23,4 +23,7 @@ source_dir=${1}/csharp
 
 pushd ${source_dir}
 dotnet test
+for pdb in artifacts/Apache.Arrow/*/*/Apache.Arrow.pdb; do
+  sourcelink test ${pdb}
+done
 popd
diff --git a/dev/release/03-binary.sh b/dev/release/03-binary.sh
index 696790c..3b845a1 100755
--- a/dev/release/03-binary.sh
+++ b/dev/release/03-binary.sh
@@ -79,6 +79,7 @@ fi
 : ${UPLOAD_CENTOS_YUM:=${UPLOAD_DEFAULT}}
 : ${UPLOAD_DEBIAN_APT:=${UPLOAD_DEFAULT}}
 : ${UPLOAD_DEBIAN_DEB:=${UPLOAD_DEFAULT}}
+: ${UPLOAD_NUGET:=${UPLOAD_DEFAULT}}
 : ${UPLOAD_PYTHON:=${UPLOAD_DEFAULT}}
 : ${UPLOAD_UBUNTU_APT:=${UPLOAD_DEFAULT}}
 : ${UPLOAD_UBUNTU_DEB:=${UPLOAD_DEFAULT}}
@@ -110,6 +111,9 @@ if [ ${UPLOAD_CENTOS_YUM} -gt 0 ]; then
   rake_tasks+=(yum:rc)
   yum_targets+=(centos)
 fi
+if [ ${UPLOAD_NUGET} -gt 0 ]; then
+  rake_tasks+=(nuget:rc)
+fi
 if [ ${UPLOAD_PYTHON} -gt 0 ]; then
   rake_tasks+=(python:rc)
 fi
diff --git a/dev/release/binary-task.rb b/dev/release/binary-task.rb
index 5ebe5ef..12433d1 100644
--- a/dev/release/binary-task.rb
+++ b/dev/release/binary-task.rb
@@ -28,8 +28,8 @@ class BinaryTask
   include Rake::DSL
 
   class ThreadPool
-    def initialize(n_workers=nil, &worker)
-      @n_workers = n_workers || detect_n_processors
+    def initialize(use_case, &worker)
+      @n_workers = choose_n_workers(use_case)
       @worker = worker
       @jobs = Thread::Queue.new
       @workers = @n_workers.times.collect do
@@ -55,11 +55,16 @@ class BinaryTask
     end
 
     private
-    def detect_n_processors
-      if File.exist?("/proc/cpuinfo")
-        File.readlines("/proc/cpuinfo").grep(/^processor/).size
+    def choose_n_workers(use_case)
+      case use_case
+      when :bintray
+        # Too many workers cause Bintray error.
+        6
+      when :gpg
+        # Too many workers cause gpg-agent error.
+        2
       else
-        8
+        raise "Unknown use case: #{use_case}"
       end
     end
   end
@@ -512,7 +517,7 @@ class BinaryTask
 
       progress_label = "Downloading: #{package} #{full_version}"
       progress_reporter = ProgressReporter.new(progress_label)
-      pool = ThreadPool.new do |path, output_path|
+      pool = ThreadPool.new(:bintray) do |path, output_path|
         download_file(path, output_path)
         progress_reporter.advance
       end
@@ -637,6 +642,7 @@ class BinaryTask
 
     def initialize(repository:,
                    distribution:,
+                   distribution_label:,
                    version:,
                    rc: nil,
                    source:,
@@ -645,6 +651,7 @@ class BinaryTask
                    api_key:)
       @repository = repository
       @distribution = distribution
+      @distribution_label = distribution_label
       @version = version
       @rc = rc
       @source = source
@@ -660,7 +667,7 @@ class BinaryTask
 
       progress_label = "Uploading: #{package} #{full_version}"
       progress_reporter = ProgressReporter.new(progress_label)
-      pool = ThreadPool.new do |path, relative_path|
+      pool = ThreadPool.new(:bintray) do |path, relative_path|
         upload_file(path, relative_path)
         progress_reporter.advance
       end
@@ -704,24 +711,20 @@ class BinaryTask
         release_type = "Release"
       end
       case @distribution
-      when "debian"
-        "#{release_type} deb packages for Debian"
-      when "ubuntu"
-        "#{release_type} deb packages for Ubuntu"
+      when "debian", "ubuntu"
+        "#{release_type} deb packages for #{@distribution_label}"
       when "centos"
-        "#{release_type} RPM packages for CentOS"
-      when "Python"
-        "#{release_type} binaries for Python"
+        "#{release_type} RPM packages for #{@distribution_label}"
       else
-        "#{release_type} binaries for #{@distribution}"
+        "#{release_type} binaries for #{@distribution_label}"
       end
     end
 
     def version_description
       if @rc
-        "Apache Arrow #{@version} RC#{@rc} for #{@distribution}"
+        "Apache Arrow #{@version} RC#{@rc} for #{@distribution_label}"
       else
-        "Apache Arrow #{@version} for #{@distribution}"
+        "Apache Arrow #{@version} for #{@distribution_label}"
       end
     end
 
@@ -779,6 +782,7 @@ class BinaryTask
     define_apt_tasks
     define_yum_tasks
     define_python_tasks
+    define_nuget_tasks
     define_summary_tasks
   end
 
@@ -1020,6 +1024,17 @@ class BinaryTask
     ]
   end
 
+  def apt_distribution_label(distribution)
+    case distribution
+    when "debian"
+      "Debian"
+    when "ubuntu"
+      "Ubuntu"
+    else
+      distribution
+    end
+  end
+
   def apt_targets
     env_apt_targets = (ENV["APT_TARGETS"] || "").split(",")
     if env_apt_targets.empty?
@@ -1124,8 +1139,10 @@ class BinaryTask
       task :upload do
         apt_distributions.each do |distribution|
           distribution_dir = "#{deb_dir}/#{distribution}"
+          distribution_label = apt_distribution_label(distribution)
           uploader = BintrayUploader.new(repository: bintray_repository,
                                          distribution: distribution,
+                                         distribution_label: distribution_label,
                                          version: version,
                                          rc: rc,
                                          source: distribution_dir,
@@ -1300,8 +1317,10 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         task :upload => apt_rc_repositories_dir do
           apt_distributions.each do |distribution|
             dists_dir = "#{apt_rc_repositories_dir}/#{distribution}/dists"
+            distribution_label = apt_distribution_label(distribution)
             uploader = BintrayUploader.new(repository: bintray_repository,
                                            distribution: distribution,
+                                           distribution_label: distribution_label,
                                            version: version,
                                            rc: rc,
                                            source: dists_dir,
@@ -1340,8 +1359,10 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         task :upload => apt_release_repositories_dir do
           apt_distributions.each do |distribution|
             distribution_dir = "#{apt_release_repositories_dir}/#{distribution}"
+            distribution_label = apt_distribution_label(distribution)
             uploader = BintrayUploader.new(repository: bintray_repository,
                                            distribution: distribution,
+                                           distribution_label: distribution_label,
                                            version: version,
                                            source: distribution_dir,
                                            user: bintray_user,
@@ -1386,6 +1407,15 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
     ]
   end
 
+  def yum_distribution_label(distribution)
+    case distribution
+    when "centos"
+      "CentOS"
+    else
+      distribution
+    end
+  end
+
   def yum_targets
     env_yum_targets = (ENV["YUM_TARGETS"] || "").split(",")
     if env_yum_targets.empty?
@@ -1420,7 +1450,7 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
   end
 
   def sign_rpms(directory)
-    thread_pool = ThreadPool.new(2) do |rpm|
+    thread_pool = ThreadPool.new(:gpg) do |rpm|
       unless signed_rpm?(rpm)
         sh("rpm",
            "-D", "_gpg_name #{gpg_key_id}",
@@ -1540,8 +1570,10 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
       task :upload do
         yum_distributions.each do |distribution|
           distribution_dir = "#{rpm_dir}/#{distribution}"
+          distribution_label = yum_distribution_label(distribution)
           uploader = BintrayUploader.new(repository: bintray_repository,
                                          distribution: distribution,
+                                         distribution_label: distribution_label,
                                          version: version,
                                          rc: rc,
                                          source: distribution_dir,
@@ -1611,6 +1643,7 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         desc "Upload RC Yum repositories"
         task :upload => yum_rc_repositories_dir do
           yum_targets.each do |distribution, distribution_version|
+            distribution_label = yum_distribution_label(distribution)
             base_dir = [
               yum_rc_repositories_dir,
               distribution,
@@ -1625,6 +1658,7 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
               uploader =
                 BintrayUploader.new(repository: bintray_repository,
                                     distribution: distribution,
+                                    distribution_label: distribution_label,
                                     version: version,
                                     rc: rc,
                                     source: repodata_dir.to_s,
@@ -1664,8 +1698,10 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         task :upload => yum_release_repositories_dir do
           yum_distributions.each do |distribution|
             distribution_dir = "#{yum_release_repositories_dir}/#{distribution}"
+            distribution_label = yum_distribution_label(distribution)
             uploader = BintrayUploader.new(repository: bintray_repository,
                                            distribution: distribution,
+                                           distribution_label: distribution_label,
                                            version: version,
                                            source: distribution_dir,
                                            user: bintray_user,
@@ -1690,28 +1726,23 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
     define_yum_release_tasks
   end
 
-  def python_rc_dir
-    "#{rc_dir}/python/#{full_version}"
-  end
-
-  def python_release_dir
-    "#{release_dir}/python/#{full_version}"
-  end
-
-  def define_python_rc_tasks
-    directory python_rc_dir
+  def define_generic_data_rc_tasks(label,
+                                   id,
+                                   rc_dir,
+                                   target_files_glob)
+    directory rc_dir
 
-    namespace :python do
+    namespace id do
       namespace :rc do
-        desc "Copy Python packages"
-        task :copy => python_rc_dir do
-          progress_label = "Copying: Python"
+        desc "Copy #{label} packages"
+        task :copy => rc_dir do
+          progress_label = "Copying: #{label}"
           progress_reporter = ProgressReporter.new(progress_label)
 
-          Pathname(artifacts_dir).glob("{conda,wheel}-*/**/*") do |path|
+          Pathname(artifacts_dir).glob(target_files_glob) do |path|
             next if path.directory?
             destination_path = [
-              python_rc_dir,
+              rc_dir,
               path.basename.to_s,
             ].join("/")
             copy_artifact(path, destination_path, progress_reporter)
@@ -1720,18 +1751,19 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
           progress_reporter.finish
         end
 
-        desc "Sign Python packages"
-        task :sign => python_rc_dir do
-          sign_dir("Python", python_rc_dir)
+        desc "Sign #{label} packages"
+        task :sign => rc_dir do
+          sign_dir(label, rc_dir)
         end
 
-        desc "Upload Python packages"
+        desc "Upload #{label} packages"
         task :upload do
           uploader = BintrayUploader.new(repository: bintray_repository,
-                                         distribution: "python",
+                                         distribution: id.to_s,
+                                         distribution_label: label,
                                          version: version,
                                          rc: rc,
-                                         source: python_rc_dir,
+                                         source: rc_dir,
                                          destination_prefix: "#{full_version}/",
                                          user: bintray_user,
                                          api_key: bintray_api_key)
@@ -1739,31 +1771,32 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         end
       end
 
-      desc "Release RC Python packages"
-      python_rc_tasks = [
-        "python:rc:copy",
-        "python:rc:sign",
-        "python:rc:upload",
+      desc "Release RC #{label} packages"
+      rc_tasks = [
+        "#{id}:rc:copy",
+        "#{id}:rc:sign",
+        "#{id}:rc:upload",
       ]
-      task :rc => python_rc_tasks
+      task :rc => rc_tasks
     end
   end
 
-  def define_python_release_tasks
-    directory python_release_dir
+  def define_generic_data_release_tasks(label, id, release_dir)
+    directory release_dir
 
-    namespace :python do
+    namespace id do
       namespace :release do
-        desc "Download RC Python packages"
-        task :download => python_release_dir do
-          download_distribution("python", python_release_dir)
+        desc "Download RC #{label} packages"
+        task :download => release_dir do
+          download_distribution(id.to_s, release_dir)
         end
 
-        desc "Upload release Python packages"
-        task :upload => python_release_dir do
-          packages_dir = "#{python_release_dir}/#{full_version}"
+        desc "Upload release #{label} packages"
+        task :upload => release_dir do
+          packages_dir = "#{release_dir}/#{full_version}"
           uploader = BintrayUploader.new(repository: bintray_repository,
-                                         distribution: "python",
+                                         distribution: id.to_s,
+                                         distribution_label: label,
                                          version: version,
                                          source: packages_dir,
                                          destination_prefix: "#{version}/",
@@ -1773,18 +1806,38 @@ APT::FTPArchive::Release::Description "#{apt_repository_description}";
         end
       end
 
-      desc "Release Python packages"
-      python_release_tasks = [
-        "python:release:download",
-        "python:release:upload",
+      desc "Release #{label} packages"
+      release_tasks = [
+        "#{id}:release:download",
+        "#{id}:release:upload",
       ]
-      task :release => python_release_tasks
+      task :release => release_tasks
     end
   end
 
+  def define_generic_data_tasks(label,
+                                id,
+                                rc_dir,
+                                release_dir,
+                                target_files_glob)
+    define_generic_data_rc_tasks(label, id, rc_dir, target_files_glob)
+    define_generic_data_release_tasks(label, id, release_dir)
+  end
+
   def define_python_tasks
-    define_python_rc_tasks
-    define_python_release_tasks
+    define_generic_data_tasks("Python",
+                              :python,
+                              "#{rc_dir}/python/#{full_version}",
+                              "#{release_dir}/python/#{full_version}",
+                              "{conda,wheel}-*/**/*")
+  end
+
+  def define_nuget_tasks
+    define_generic_data_tasks("NuGet",
+                              :nuget,
+                              "#{rc_dir}/nuget/#{full_version}",
+                              "#{release_dir}/nuget/#{full_version}",
+                              "nuget/**/*")
   end
 
   def define_summary_tasks
@@ -1797,6 +1850,7 @@ Success! The release candidate binaries are available here:
   https://bintray.com/#{bintray_repository}/ubuntu-rc/#{full_version}
   https://bintray.com/#{bintray_repository}/centos-rc/#{full_version}
   https://bintray.com/#{bintray_repository}/python-rc/#{full_version}
+  https://bintray.com/#{bintray_repository}/nuget-rc/#{full_version}
         SUMMARY
       end
 
@@ -1808,6 +1862,7 @@ Success! The release binaries are available here:
   https://bintray.com/#{bintray_repository}/ubuntu/#{version}
   https://bintray.com/#{bintray_repository}/centos/#{version}
   https://bintray.com/#{bintray_repository}/python/#{version}
+  https://bintray.com/#{bintray_repository}/nuget/#{version}
         SUMMARY
       end
     end
diff --git a/dev/release/download_rc_binaries.py b/dev/release/download_rc_binaries.py
index 843f0c0..1c3da0e 100755
--- a/dev/release/download_rc_binaries.py
+++ b/dev/release/download_rc_binaries.py
@@ -31,7 +31,7 @@ import urllib.request
 
 BINTRAY_API_ROOT = "https://bintray.com/api/v1"
 BINTRAY_DL_ROOT = "https://dl.bintray.com"
-BINTRAY_REPO = 'apache/arrow'
+BINTRAY_REPO = os.getenv('BINTRAY_REPOSITORY', 'apache/arrow')
 DEFAULT_PARALLEL_DOWNLOADS = 8
 
 
@@ -131,7 +131,7 @@ def parallel_map_terminate_early(f, iterable, num_parallel):
                 raise e
 
 
-ARROW_PACKAGE_TYPES = ['centos', 'debian', 'python', 'ubuntu']
+ARROW_PACKAGE_TYPES = ['centos', 'debian', 'nuget', 'python', 'ubuntu']
 
 
 def download_rc_binaries(version, rc_number, re_match=None, dest=None,
diff --git a/dev/release/post-02-binary.sh b/dev/release/post-02-binary.sh
index b07abb7..9f531af 100755
--- a/dev/release/post-02-binary.sh
+++ b/dev/release/post-02-binary.sh
@@ -53,6 +53,7 @@ fi
 : ${DEPLOY_DEFAULT:=1}
 : ${DEPLOY_CENTOS:=${DEPLOY_DEFAULT}}
 : ${DEPLOY_DEBIAN:=${DEPLOY_DEFAULT}}
+: ${DEPLOY_NUGET:=${DEPLOY_DEFAULT}}
 : ${DEPLOY_PYTHON:=${DEPLOY_DEFAULT}}
 : ${DEPLOY_UBUNTU:=${DEPLOY_DEFAULT}}
 
@@ -71,6 +72,9 @@ if [ ${DEPLOY_CENTOS} -gt 0 ]; then
   rake_tasks+=(yum:release)
   yum_targets+=(centos)
 fi
+if [ ${DEPLOY_NUGET} -gt 0 ]; then
+  rake_tasks+=(nuget:release)
+fi
 if [ ${DEPLOY_PYTHON} -gt 0 ]; then
   rake_tasks+=(python:release)
 fi
diff --git a/dev/release/post-06-csharp.sh b/dev/release/post-06-csharp.sh
index 2fc5ca0..7b28378 100755
--- a/dev/release/post-06-csharp.sh
+++ b/dev/release/post-06-csharp.sh
@@ -18,7 +18,8 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-set -e
+
+set -eux
 
 SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)"
 
@@ -34,28 +35,20 @@ if [ -z "${NUGET_API_KEY}" ]; then
   exit 1
 fi
 
-archive_name=apache-arrow-${version}
-tar_gz=${archive_name}.tar.gz
-rm -f ${tar_gz}
-curl \
-  --remote-name \
-  --fail \
-  https://downloads.apache.org/arrow/arrow-${version}/${tar_gz}
-rm -rf ${archive_name}
-tar xf ${tar_gz}
-pushd ${archive_name}/csharp
-mv dummy.git ../.git
-dotnet pack -c Release
-mv ../.git dummy.git
-for package in artifacts/Apache.Arrow/Release/*.{nupkg,snupkg}; do
-  dotnet nuget push \
-    ${package} \
-    -k ${NUGET_API_KEY} \
-    -s https://api.nuget.org/v3/index.json
+base_name=Apache.Arrow.${version}
+for extension in nupkg snupkg; do
+  path=${base_name}.${extension}
+  rm -f ${path}
+  curl \
+    --fail \
+    --location \
+    --remote-name \
+    https://apache.bintray.com/arrow/nuget/${version}/${path}
 done
-popd
-rm -rf ${archive_name}
-rm -f ${tar_gz}
+dotnet nuget push \
+  ${base_name}.nupkg \
+  -k ${NUGET_API_KEY} \
+  -s https://api.nuget.org/v3/index.json
 
 echo "Success! The released NuGet package is available here:"
 echo "  https://www.nuget.org/packages/Apache.Arrow/${version}"
diff --git a/dev/tasks/crossbow.py b/dev/tasks/crossbow.py
index cd91090..52e4571 100755
--- a/dev/tasks/crossbow.py
+++ b/dev/tasks/crossbow.py
@@ -799,6 +799,19 @@ class Target(Serializable):
         self.remote = remote
         self.version = version
         self.no_rc_version = re.sub(r'-rc\d+\Z', '', version)
+        # Semantic Versioning 1.0.0: https://semver.org/spec/v1.0.0.html
+        #
+        # > A pre-release version number MAY be denoted by appending an
+        # > arbitrary string immediately following the patch version and a
+        # > dash. The string MUST be comprised of only alphanumerics plus
+        # > dash [0-9A-Za-z-].
+        #
+        # Example:
+        #
+        #   '0.16.1.dev10' ->
+        #   '0.16.1-dev10'
+        self.no_rc_semver_version = \
+            re.sub(r'\.(dev\d+)\Z', r'-\1', self.no_rc_version)
 
     @classmethod
     def from_repo(cls, repo, head=None, branch=None, remote=None, version=None,
@@ -987,7 +1000,8 @@ class Job(Serializable):
         # instantiate the tasks
         tasks = {}
         versions = {'version': target.version,
-                    'no_rc_version': target.no_rc_version}
+                    'no_rc_version': target.no_rc_version,
+                    'no_rc_semver_version': target.no_rc_semver_version}
         for task_name, task in task_definitions.items():
             artifacts = task.pop('artifacts', None) or []  # because of yaml
             artifacts = [fn.format(**versions) for fn in artifacts]
diff --git a/dev/tasks/nuget-packages/github.linux.yml b/dev/tasks/nuget-packages/github.linux.yml
new file mode 100644
index 0000000..c8143e4
--- /dev/null
+++ b/dev/tasks/nuget-packages/github.linux.yml
@@ -0,0 +1,82 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# NOTE: must set "Crossbow" as name to have the badge links working in the
+# GitHub comment reports!
+name: Crossbow
+
+on:
+  push:
+    branches:
+      - "*-github-*"
+
+env:
+  DOCKER_BUILDKIT: 0
+  COMPOSE_DOCKER_CLI_BUILD: 1
+
+jobs:
+  package:
+    name: Package
+    runs-on: ubuntu-latest
+    steps:
+      - name: Checkout Arrow
+        run: |
+          git clone --no-checkout {{ arrow.remote }} arrow
+          git -C arrow fetch -t {{ arrow.remote }} {{ arrow.branch }}
+          git -C arrow checkout FETCH_HEAD
+          git -C arrow submodule update --init --recursive
+      - name: Setup Python
+        uses: actions/setup-python@v1
+      - name: Setup Archery
+        run: pip install -e arrow/dev/archery[docker]
+      - name: Prepare version
+        run: |
+          sed -i'' -E -e \
+            "s/^    <Version>.+<\/Version>/    <Version>{{ arrow.no_rc_semver_version }}<\/Version>/" \
+            arrow/csharp/Directory.Build.props
+      - name: Build package
+        run: |
+          pushd arrow
+          archery docker run {{ run }}
+          popd
+      # Using GitHub release tries to find a common ancestor between the
+      # currently pushed tag and the latest tag of the GitHub repository
+      # (don't know why).
+      # The tag upload took 43 minutes because of this scan, so use an
+      # alternative upload script.
+      - name: Set up Crossbow
+        run: |
+          pip install \
+            click \
+            github3.py \
+            jinja2 \
+            jira \
+            pygit2 \
+            ruamel.yaml \
+            setuptools_scm \
+            toolz
+      - name: Upload artifacts
+        run: |
+          python arrow/dev/tasks/crossbow.py \
+            --queue-path . \
+            --queue-remote {{ queue_remote_url }} \
+            upload-artifacts \
+            --pattern "arrow/csharp/artifacts/Apache.Arrow/Release/Apache.Arrow.*" \
+            --sha {{ task.branch }} \
+            --tag {{ task.tag }}
+        env:
+          CROSSBOW_GITHUB_TOKEN: {{ '${{ secrets.CROSSBOW_GITHUB_TOKEN }}' }}
diff --git a/dev/tasks/tasks.yml b/dev/tasks/tasks.yml
index 8ff78ee..b50fe4c 100644
--- a/dev/tasks/tasks.yml
+++ b/dev/tasks/tasks.yml
@@ -96,7 +96,10 @@ groups:
     - ubuntu-*-amd64
     - conda-*
     - gandiva-*
-    - homebrew-*
+    # List the homebrews explicitly because we don't care about running homebrew-cpp-autobrew
+    - homebrew-cpp
+    - homebrew-r-autobrew
+    - nuget
     - test-*
     - wheel-*
 
@@ -1420,6 +1423,17 @@ tasks:
     artifacts:
       - arrow-gandiva-{no_rc_version}-SNAPSHOT.jar
 
+  ############################## NuGet packages ###############################
+
+  nuget:
+    ci: github
+    template: nuget-packages/github.linux.yml
+    params:
+      run: ubuntu-csharp
+    artifacts:
+      - Apache.Arrow.{no_rc_version}.nupkg
+      - Apache.Arrow.{no_rc_version}.snupkg
+
   ########################### Release verification ############################
 
   verify-rc-binaries:
diff --git a/docker-compose.yml b/docker-compose.yml
index ec8d483..5422d1d 100644
--- a/docker-compose.yml
+++ b/docker-compose.yml
@@ -941,8 +941,8 @@ services:
 
   ubuntu-csharp:
     # Usage:
-    #   docker-compose build debian-csharp
-    #   docker-compose run debian-csharp
+    #   docker-compose build ubuntu-csharp
+    #   docker-compose run ubuntu-csharp
     image: ${REPO}:${ARCH}-ubuntu-18.04-csharp-${DOTNET}
     build:
       context: .
@@ -957,7 +957,8 @@ services:
     command: &csharp-command >
       /bin/bash -c "
         /arrow/ci/scripts/csharp_build.sh /arrow &&
-        /arrow/ci/scripts/csharp_test.sh /arrow"
+        /arrow/ci/scripts/csharp_test.sh /arrow &&
+        /arrow/ci/scripts/csharp_pack.sh /arrow"
 
   ################################ Java #######################################